From d46d2170f8f9f9a9444d81c702ac5f48bc5ba192 Mon Sep 17 00:00:00 2001 From: Apekshit Sharma Date: Mon, 23 Oct 2017 17:01:27 -0700 Subject: [PATCH] HBASE-19114 Split out o.a.h.h.zookeeper from hbase-server and hbase-client Change-Id: I60c20f48f98c95246e22f26eb7df3b1a402a7ace --- hbase-assembly/pom.xml | 4 + .../src/main/assembly/hadoop-two-compat.xml | 1 + hbase-client/pom.xml | 6 +- .../hadoop/hbase/client/ClusterConnection.java | 2 +- .../hbase/client/ConnectionImplementation.java | 6 +- .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 2 +- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 4 - .../org/apache/hadoop/hbase/client/RegionInfo.java | 2 +- .../hadoop/hbase/client/ZooKeeperRegistry.java | 4 +- .../hadoop/hbase/ipc/CoprocessorRpcUtils.java | 4 +- .../hbase/ipc/RemoteWithExtrasException.java | 3 +- .../hbase/security/access/AccessControlClient.java | 2 +- .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java | 10 +- .../hbase/zookeeper/MasterAddressTracker.java | 18 +- .../hadoop/hbase/zookeeper/MetaTableLocator.java | 13 +- .../apache/hadoop/hbase/zookeeper/ZKClusterId.java | 8 +- .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java | 1868 +------------------ hbase-common/pom.xml | 4 +- .../java/org/apache/hadoop/hbase/Abortable.java | 0 .../java/org/apache/hadoop/hbase/HConstants.java | 2 + .../hadoop/hbase/protobuf/ProtobufHelpers.java | 25 + .../hadoop/hbase/protobuf/ProtobufMagic.java | 52 +- .../regionserver/SecureBulkLoadEndpointClient.java | 1 - hbase-examples/pom.xml | 4 + hbase-it/pom.xml | 5 + .../hadoop/hbase/DistributedHBaseCluster.java | 1 + .../hadoop/hbase/IntegrationTestMetaReplicas.java | 10 +- .../test/IntegrationTestZKAndFSPermissions.java | 6 +- .../org/apache/hadoop/hbase/mapreduce/Import.java | 2 +- hbase-replication/pom.xml | 4 + .../hbase/replication/ReplicationPeerZKImpl.java | 8 +- .../hbase/replication/ReplicationPeersZKImpl.java | 42 +- .../replication/ReplicationQueuesClientZKImpl.java | 26 +- .../hbase/replication/ReplicationQueuesZKImpl.java | 123 +- .../hbase/replication/ReplicationStateZKBase.java | 20 +- .../replication/ReplicationTrackerZKImpl.java | 4 +- .../hbase/rsgroup/RSGroupInfoManagerImpl.java | 36 +- .../hbase/rsgroup/VerifyingRSGroupAdminClient.java | 8 +- hbase-server/pom.xml | 10 + .../apache/hadoop/hbase/ZKNamespaceManager.java | 42 +- .../java/org/apache/hadoop/hbase/ZNodeClearer.java | 18 +- .../hbase/backup/example/HFileArchiveManager.java | 18 +- .../backup/example/TableHFileArchiveTracker.java | 18 +- .../hbase/backup/example/ZKTableArchiveClient.java | 4 +- .../ZKSplitLogManagerCoordination.java | 66 +- .../coordination/ZkSplitLogWorkerCoordination.java | 62 +- .../hadoop/hbase/master/ActiveMasterManager.java | 18 +- .../DrainingServerTracker.java | 15 +- .../org/apache/hadoop/hbase/master/HMaster.java | 21 +- .../hadoop/hbase/master/HMasterCommandLine.java | 6 +- .../hadoop/hbase/master/MasterMetaBootstrap.java | 4 +- .../{zookeeper => master}/RegionServerTracker.java | 21 +- .../apache/hadoop/hbase/master/ServerManager.java | 7 +- .../{zookeeper => master}/SplitOrMergeTracker.java | 29 +- .../master/cleaner/ReplicationZKNodeCleaner.java | 13 +- .../hbase/procedure/ZKProcedureCoordinator.java | 34 +- .../hbase/procedure/ZKProcedureMemberRpcs.java | 37 +- .../hadoop/hbase/procedure/ZKProcedureUtil.java | 37 +- .../hadoop/hbase/regionserver/HRegionServer.java | 32 +- .../RecoveringRegionWatcher.java | 8 +- .../replication/HBaseReplicationEndpoint.java | 4 +- .../master/ReplicationHFileCleaner.java | 1 - .../hbase/replication/master/TableCFsUpdater.java | 12 +- .../hbase/security/access/AccessControlLists.java | 3 +- .../hbase/security/access/ZKPermissionWatcher.java | 55 +- .../token/AuthenticationTokenSecretManager.java | 6 +- .../hbase/security/token/ZKSecretWatcher.java | 54 +- .../hbase/security/visibility/VisibilityUtils.java | 4 +- .../visibility/ZKVisibilityLabelWatcher.java | 24 +- .../org/apache/hadoop/hbase/util/HBaseFsck.java | 12 +- .../apache/hadoop/hbase/util/HBaseFsckRepair.java | 2 +- .../apache/hadoop/hbase/util/ZKDataMigrator.java | 10 +- .../apache/hadoop/hbase/HBaseTestingUtility.java | 1 + .../org/apache/hadoop/hbase/TestMultiVersions.java | 7 +- .../org/apache/hadoop/hbase/TestZooKeeper.java | 36 +- .../example/TestZooKeeperTableArchiveClient.java | 4 +- .../hbase/client/HConnectionTestingUtility.java | 3 +- .../org/apache/hadoop/hbase/client/TestAdmin2.java | 9 +- .../hbase/client/TestHBaseAdminNoCluster.java | 6 +- .../hadoop/hbase/client/TestMetaWithReplicas.java | 22 +- .../hadoop/hbase/filter/FilterTestingCluster.java | 2 +- .../hadoop/hbase/filter/TestFilterWrapper.java | 2 +- .../hadoop/hbase/master/MockRegionServer.java | 5 +- .../hbase/master/TestActiveMasterManager.java | 20 +- .../hbase/master/TestAssignmentListener.java | 10 +- .../hbase/master/TestDistributedLogSplitting.java | 8 +- .../hbase/master/TestHMasterRPCException.java | 4 +- .../hadoop/hbase/master/TestMasterNoCluster.java | 6 +- .../hadoop/hbase/master/TestMasterWalManager.java | 24 +- .../hbase/master/TestMetaShutdownHandler.java | 6 +- .../hadoop/hbase/master/TestSplitLogManager.java | 71 +- .../hadoop/hbase/master/TestTableStateManager.java | 10 +- .../hbase/master/cleaner/TestLogsCleaner.java | 2 +- .../cleaner/TestReplicationHFileCleaner.java | 2 +- .../procedure/TestZKProcedureControllers.java | 20 +- .../TestCompactionInDeadRegionServer.java | 4 +- .../regionserver/TestMasterAddressTracker.java | 4 +- .../regionserver/TestRegionServerHostname.java | 6 +- .../hbase/regionserver/TestSplitLogWorker.java | 45 +- .../TestSplitTransactionOnCluster.java | 4 +- .../hbase/replication/TestMasterReplication.java | 7 +- .../replication/TestReplicationStateHBaseImpl.java | 6 +- .../replication/TestReplicationStateZKImpl.java | 15 +- .../replication/TestReplicationTrackerZKImpl.java | 38 +- .../replication/master/TestTableCFsUpdater.java | 41 +- .../regionserver/TestReplicationSourceManager.java | 16 +- .../security/access/TestAccessController2.java | 12 +- .../token/TestZKSecretWatcherRefreshKeys.java | 14 +- .../org/apache/hadoop/hbase/util/MockServer.java | 4 +- .../hbase/util/ProcessBasedLocalHBaseCluster.java | 4 +- .../hbase/zookeeper/TestRecoverableZooKeeper.java | 2 +- .../hbase/zookeeper/TestZKLeaderManager.java | 6 +- .../apache/hadoop/hbase/zookeeper/TestZKMulti.java | 226 +-- .../hadoop/hbase/zookeeper/TestZooKeeperACL.java | 20 +- .../hbase/zookeeper/TestZooKeeperMainServer.java | 6 +- .../hbase/zookeeper/TestZooKeeperNodeTracker.java | 22 +- hbase-zookeeper/pom.xml | 655 +++++++ .../hbase/zookeeper/ClusterStatusTracker.java | 14 +- .../hadoop/hbase/zookeeper/DeletionListener.java | 2 +- .../hadoop/hbase/zookeeper/EmptyWatcher.java | 0 .../apache/hadoop/hbase/zookeeper/HQuorumPeer.java | 2 +- .../hadoop/hbase/zookeeper/InstancePending.java | 0 .../hbase/zookeeper/LoadBalancerTracker.java | 15 +- .../zookeeper/MasterMaintenanceModeTracker.java | 4 +- .../hadoop/hbase/zookeeper/MetricsZooKeeper.java | 0 .../hbase/zookeeper/MiniZooKeeperCluster.java | 0 .../hadoop/hbase/zookeeper/PendingWatcher.java | 0 .../hbase/zookeeper/RecoverableZooKeeper.java | 0 .../hbase/zookeeper/RegionNormalizerTracker.java | 15 +- .../hadoop/hbase/zookeeper/ZKLeaderManager.java | 18 +- .../hadoop/hbase/zookeeper/ZKServerTool.java | 0 .../apache/hadoop/hbase/zookeeper/ZKSplitLog.java | 76 +- .../apache/hadoop/hbase/zookeeper/ZNodePaths.java | 54 +- .../apache/hadoop/hbase/zookeeper/ZkAclReset.java | 6 +- .../zookeeper}/ZooKeeperConnectionException.java | 2 +- .../hadoop/hbase/zookeeper/ZooKeeperListener.java | 0 .../hbase/zookeeper/ZooKeeperMainServer.java | 0 .../hbase/zookeeper/ZooKeeperMetricsListener.java | 0 .../hbase/zookeeper/ZooKeeperNodeTracker.java | 20 +- .../hadoop/hbase/zookeeper/ZooKeeperUtil.java | 1886 ++++++++++++++++++++ .../hadoop/hbase/zookeeper/ZooKeeperWatcher.java | 35 +- .../hbase/zookeeper/TestInstancePending.java | 0 .../hbase/zookeeper/TestMetricsZooKeeper.java | 0 .../apache/hadoop/hbase/zookeeper/TestZKUtil.java | 11 +- .../hbase/zookeeper/TestZooKeeperWatcher.java | 3 +- pom.xml | 13 + 146 files changed, 3720 insertions(+), 2940 deletions(-) rename {hbase-client => hbase-common}/src/main/java/org/apache/hadoop/hbase/Abortable.java (100%) create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufHelpers.java rename {hbase-client => hbase-common}/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java (66%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{zookeeper => master}/DrainingServerTracker.java (88%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{zookeeper => master}/RegionServerTracker.java (88%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{zookeeper => master}/SplitOrMergeTracker.java (84%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{zookeeper => regionserver}/RecoveringRegionWatcher.java (91%) create mode 100644 hbase-zookeeper/pom.xml rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java (88%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java (97%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java (100%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java (98%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java (100%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java (85%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java (93%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java (100%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java (100%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java (100%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java (100%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java (85%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java (90%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java (100%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java (62%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java (79%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java (94%) rename {hbase-client/src/main/java/org/apache/hadoop/hbase => hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper}/ZooKeeperConnectionException.java (97%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java (100%) rename {hbase-server => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java (100%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java (100%) rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (92%) create mode 100644 hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperUtil.java rename {hbase-client => hbase-zookeeper}/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (94%) rename {hbase-client => hbase-zookeeper}/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java (100%) rename {hbase-client => hbase-zookeeper}/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java (100%) rename {hbase-client => hbase-zookeeper}/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java (93%) rename {hbase-client => hbase-zookeeper}/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java (94%) diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml index 18b63b5414..5d06e44d83 100644 --- a/hbase-assembly/pom.xml +++ b/hbase-assembly/pom.xml @@ -191,6 +191,10 @@ hbase-it test-jar + + org.apache.hbase + hbase-zookeeper + org.apache.hbase hbase-server diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml index a66237bf0d..c2e1480572 100644 --- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml +++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml @@ -51,6 +51,7 @@ org.apache.hbase:hbase-external-blockcache org.apache.hbase:hbase-backup org.apache.hbase:hbase-mapreduce + org.apache.hbase:hbase-zookeeper diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index a8e73c7cb5..789fc13f3c 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -102,6 +102,10 @@ org.apache.hbase hbase-common + + org.apache.hbase + hbase-zookeeper + org.apache.hbase hbase-hadoop-compat @@ -285,7 +289,7 @@ com.sun.jersey jersey-json - + javax.servlet servlet-api diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index a931b1dcb9..9c231ddadf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; 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 aa696121c3..b236c7905f 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 @@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; @@ -73,7 +73,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; import org.apache.yetus.audience.InterfaceAudience; @@ -539,7 +539,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { throws MasterNotRunningException { String errorMsg; try { - if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) { + if (ZooKeeperUtil.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'. " 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 e4bb675bb4..02fbe58b73 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 @@ -69,7 +69,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper; import org.apache.hadoop.hbase.client.replication.TableCFs; import org.apache.hadoop.hbase.client.security.SecurityCapability; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 0835a9b87e..10fc671cc1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -26,7 +26,6 @@ import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -82,7 +81,6 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ForeignExceptionUtil; -import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback; @@ -156,8 +154,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java index 0eb4e42214..0955fffb85 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java @@ -79,7 +79,7 @@ public interface RegionInfo { int MD5_HEX_LENGTH = 32; @InterfaceAudience.Private - int DEFAULT_REPLICA_ID = 0; + int DEFAULT_REPLICA_ID = HConstants.DEFAULT_REPLICA_ID; /** * to keep appended int's sorted in string format. Only allows 2 bytes 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 746382f850..2d17a9ee98 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 @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -119,7 +119,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.znodePaths.rsZNode); + return ZooKeeperUtil.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/ipc/CoprocessorRpcUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java index 4f0e5e65d0..532bc06435 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcUtils.java @@ -27,10 +27,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; @@ -124,7 +124,7 @@ public final class CoprocessorRpcUtils { throws IOException { Message.Builder builderForType = service.getRequestPrototype(methodDesc).newBuilderForType(); - org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builderForType, + ProtobufUtil.mergeFrom(builderForType, // TODO: COPY FROM SHADED TO NON_SHADED. DO I HAVE TOO? shadedRequest.toByteArray()); return builderForType.build(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java index 1374ab0430..b6bca2be74 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RemoteWithExtrasException.java @@ -24,13 +24,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.DynamicClassLoader; import org.apache.hadoop.ipc.RemoteException; /** * A {@link RemoteException} with some extra information. If source exception - * was a {@link org.apache.hadoop.hbase.DoNotRetryIOException}, + * was a {@link org.apache.hadoop.hbase.DoNotRetryIOException}, * {@link #isDoNotRetry()} will return true. *

A {@link RemoteException} hosts exceptions we got from the server. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java index 0363ba269d..94c191f94f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index c7450b4a71..c8b3c093e5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -43,8 +43,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ByteBufferCell; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellBuilder; -import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -81,7 +79,6 @@ import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionLoadStats; -import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.SnapshotDescription; @@ -287,7 +284,7 @@ public final class ProtobufUtil { * byte array that is bytes.length plus {@link ProtobufMagic#PB_MAGIC}.length. */ public static byte [] prependPBMagic(final byte [] bytes) { - return Bytes.add(ProtobufMagic.PB_MAGIC, bytes); + return ProtobufMagic.prependPBMagic(bytes); } /** @@ -313,10 +310,7 @@ public final class ProtobufUtil { * @throws DeserializationException if we are missing the pb magic prefix */ public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException { - if (!isPBMagicPrefix(bytes)) { - throw new DeserializationException("Missing pb magic " + - Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix"); - } + ProtobufMagic.expectPBMagicPrefix(bytes); } /** 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 04e26629f3..a079ab5e49 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 @@ -100,10 +100,10 @@ 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.znodePaths.backupMasterAddressesZNode, + String backupZNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.backupMasterAddressesZNode, sn.toString()); try { - byte[] data = ZKUtil.getData(watcher, backupZNode); + byte[] data = ZooKeeperUtil.getData(watcher, backupZNode); final ZooKeeperProtos.Master backup = parse(data); if (backup == null) { return 0; @@ -139,14 +139,14 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * @param zkw ZooKeeperWatcher to use * @return ServerName stored in the the master address znode or null if no * znode present. - * @throws KeeperException - * @throws IOException + * @throws KeeperException + * @throws IOException */ public static ServerName getMasterAddress(final ZooKeeperWatcher zkw) throws KeeperException, IOException { byte [] data; try { - data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); + data = ZooKeeperUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); } catch (InterruptedException e) { throw new InterruptedIOException(); } @@ -178,7 +178,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { IOException { byte[] data; try { - data = ZKUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); + data = ZooKeeperUtil.getData(zkw, zkw.znodePaths.masterAddressZNode); } catch (InterruptedException e) { throw new InterruptedIOException(); } @@ -209,7 +209,7 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { public static boolean setMasterAddress(final ZooKeeperWatcher zkw, final String znode, final ServerName master, int infoPort) throws KeeperException { - return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort)); + return ZooKeeperUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort)); } /** @@ -265,10 +265,10 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { try { Stat stat = new Stat(); - byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat); + byte[] data = ZooKeeperUtil.getDataNoWatch(zkw, zkw.znodePaths.masterAddressZNode, stat); ServerName sn = ProtobufUtil.parseServerNameFrom(data); if (sn != null && content.equals(sn.toString())) { - return (ZKUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion())); + return (ZooKeeperUtil.deleteNode(zkw, zkw.znodePaths.masterAddressZNode, stat.getVersion())); } } catch (KeeperException e) { LOG.warn("Can't get or delete the master znode", e); 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 cddde2f521..e3c44db45d 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 @@ -205,7 +205,7 @@ public class MetaTableLocator { public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout) throws InterruptedException, NotAllMetaRegionsOnlineException { try { - if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) { + if (ZooKeeperUtil.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); @@ -454,7 +454,7 @@ public class MetaTableLocator { .setState(state.convert()).build(); byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray()); try { - ZKUtil.setData(zookeeper, + ZooKeeperUtil.setData(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data); } catch(KeeperException.NoNodeException nne) { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { @@ -463,7 +463,8 @@ public class MetaTableLocator { LOG.debug("META region location doesn't exist for replicaId=" + replicaId + ", create it"); } - ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data); + ZooKeeperUtil + .createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data); } } @@ -486,7 +487,7 @@ public class MetaTableLocator { RegionState.State state = RegionState.State.OPEN; ServerName serverName = null; try { - byte[] data = ZKUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId)); + byte[] data = ZooKeeperUtil.getData(zkw, zkw.znodePaths.getZNodeForReplica(replicaId)); if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { try { int prefixLen = ProtobufUtil.lengthOfPBMagic(); @@ -507,7 +508,7 @@ public class MetaTableLocator { serverName = ProtobufUtil.parseServerNameFrom(data); } } catch (DeserializationException e) { - throw ZKUtil.convert(e); + throw ZooKeeperUtil.convert(e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } @@ -538,7 +539,7 @@ public class MetaTableLocator { } try { // Just delete the node. Don't need any watches. - ZKUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId)); + ZooKeeperUtil.deleteNode(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId)); } catch(KeeperException.NoNodeException nne) { // Has already been deleted } 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 9ef7691806..50ea87dc96 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.znodePaths.clusterIdZNode) != -1) { + if (ZooKeeperUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) { byte [] data; try { - data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode); + data = ZooKeeperUtil.getData(watcher, watcher.znodePaths.clusterIdZNode); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; @@ -74,7 +74,7 @@ public class ZKClusterId { try { return ClusterId.parseFrom(data).toString(); } catch (DeserializationException e) { - throw ZKUtil.convert(e); + throw ZooKeeperUtil.convert(e); } } } @@ -83,7 +83,7 @@ public class ZKClusterId { public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id) throws KeeperException { - ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray()); + ZooKeeperUtil.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 060792fd21..05dfb2606e 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 @@ -18,33 +18,18 @@ */ package org.apache.hadoop.hbase.zookeeper; -import java.io.BufferedReader; import java.io.IOException; -import java.io.InputStreamReader; -import java.io.PrintWriter; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Deque; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; -import javax.security.auth.login.AppConfigurationEntry; -import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; - -import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -52,30 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.authentication.util.KerberosUtil; -import org.apache.zookeeper.AsyncCallback; -import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.Op; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooDefs.Ids; -import org.apache.zookeeper.ZooDefs.Perms; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.client.ZooKeeperSaslClient; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Id; -import org.apache.zookeeper.data.Stat; -import org.apache.zookeeper.proto.CreateRequest; -import org.apache.zookeeper.proto.DeleteRequest; -import org.apache.zookeeper.proto.SetDataRequest; -import org.apache.zookeeper.server.ZooKeeperSaslServer; /** * Internal HBase utility class for ZooKeeper. @@ -89,1621 +51,11 @@ import org.apache.zookeeper.server.ZooKeeperSaslServer; @InterfaceAudience.Private public class ZKUtil { private static final Log LOG = LogFactory.getLog(ZKUtil.class); - - // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved. - public static final char ZNODE_PATH_SEPARATOR = '/'; - private static int zkDumpConnectionTimeOut; - - /** - * Creates a new connection to ZooKeeper, pulling settings and ensemble config - * from the specified configuration object using methods from {@link ZKConfig}. - * - * Sets the connection status monitoring watcher to the specified watcher. - * - * @param conf configuration to pull ensemble and other settings from - * @param watcher watcher to monitor connection changes - * @return connection to zookeeper - * @throws IOException if unable to connect to zk or config problem - */ - public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher) - throws IOException { - String ensemble = ZKConfig.getZKQuorumServersString(conf); - return connect(conf, ensemble, watcher); - } - - public static RecoverableZooKeeper connect(Configuration conf, String ensemble, - Watcher watcher) - throws IOException { - return connect(conf, ensemble, watcher, null); - } - - public static RecoverableZooKeeper connect(Configuration conf, String ensemble, - Watcher watcher, final String identifier) - throws IOException { - if(ensemble == null) { - throw new IOException("Unable to determine ZooKeeper ensemble"); - } - int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, - HConstants.DEFAULT_ZK_SESSION_TIMEOUT); - if (LOG.isTraceEnabled()) { - LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble); - } - int retry = conf.getInt("zookeeper.recovery.retry", 3); - int retryIntervalMillis = - conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); - int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000); - zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout", - 1000); - return new RecoverableZooKeeper(ensemble, timeout, watcher, - retry, retryIntervalMillis, maxSleepTime, identifier); - } - - /** - * Log in the current zookeeper server process using the given configuration - * keys for the credential file and login principal. - * - *

This is only applicable when running on secure hbase - * On regular HBase (without security features), this will safely be ignored. - *

- * - * @param conf The configuration data to use - * @param keytabFileKey Property key used to configure the path to the credential file - * @param userNameKey Property key used to configure the login principal - * @param hostname Current hostname to use in any credentials - * @throws IOException underlying exception from SecurityUtil.login() call - */ - public static void loginServer(Configuration conf, String keytabFileKey, - String userNameKey, String hostname) throws IOException { - login(conf, keytabFileKey, userNameKey, hostname, - ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY, - JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME); - } - - /** - * Log in the current zookeeper client using the given configuration - * keys for the credential file and login principal. - * - *

This is only applicable when running on secure hbase - * On regular HBase (without security features), this will safely be ignored. - *

- * - * @param conf The configuration data to use - * @param keytabFileKey Property key used to configure the path to the credential file - * @param userNameKey Property key used to configure the login principal - * @param hostname Current hostname to use in any credentials - * @throws IOException underlying exception from SecurityUtil.login() call - */ - public static void loginClient(Configuration conf, String keytabFileKey, - String userNameKey, String hostname) throws IOException { - login(conf, keytabFileKey, userNameKey, hostname, - ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, - JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME); - } - - /** - * Log in the current process using the given configuration keys for the - * credential file and login principal. - * - *

This is only applicable when running on secure hbase - * On regular HBase (without security features), this will safely be ignored. - *

- * - * @param conf The configuration data to use - * @param keytabFileKey Property key used to configure the path to the credential file - * @param userNameKey Property key used to configure the login principal - * @param hostname Current hostname to use in any credentials - * @param loginContextProperty property name to expose the entry name - * @param loginContextName jaas entry name - * @throws IOException underlying exception from SecurityUtil.login() call - */ - private static void login(Configuration conf, String keytabFileKey, - String userNameKey, String hostname, - String loginContextProperty, String loginContextName) - throws IOException { - if (!isSecureZooKeeper(conf)) - return; - - // User has specified a jaas.conf, keep this one as the good one. - // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf" - if (System.getProperty("java.security.auth.login.config") != null) - return; - - // No keytab specified, no auth - String keytabFilename = conf.get(keytabFileKey); - if (keytabFilename == null) { - LOG.warn("no keytab specified for: " + keytabFileKey); - return; - } - - String principalConfig = conf.get(userNameKey, System.getProperty("user.name")); - String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname); - - // Initialize the "jaas.conf" for keyTab/principal, - // If keyTab is not specified use the Ticket Cache. - // and set the zookeeper login context name. - JaasConfiguration jaasConf = new JaasConfiguration(loginContextName, - principalName, keytabFilename); - javax.security.auth.login.Configuration.setConfiguration(jaasConf); - System.setProperty(loginContextProperty, loginContextName); - } - - /** - * A JAAS configuration that defines the login modules that we want to use for login. - */ - private static class JaasConfiguration extends javax.security.auth.login.Configuration { - private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME = - "zookeeper-server-keytab-kerberos"; - private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME = - "zookeeper-client-keytab-kerberos"; - - private static final Map BASIC_JAAS_OPTIONS = new HashMap<>(); - static { - String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - BASIC_JAAS_OPTIONS.put("debug", "true"); - } - } - - private static final Map KEYTAB_KERBEROS_OPTIONS = new HashMap<>(); - static { - KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true"); - KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true"); - KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true"); - KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS); - } - - private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN = - new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(), - LoginModuleControlFlag.REQUIRED, - KEYTAB_KERBEROS_OPTIONS); - - private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF = - new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN}; - - private javax.security.auth.login.Configuration baseConfig; - private final String loginContextName; - private final boolean useTicketCache; - private final String keytabFile; - private final String principal; - - public JaasConfiguration(String loginContextName, String principal, String keytabFile) { - this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0); - } - - private JaasConfiguration(String loginContextName, String principal, - String keytabFile, boolean useTicketCache) { - try { - this.baseConfig = javax.security.auth.login.Configuration.getConfiguration(); - } catch (SecurityException e) { - this.baseConfig = null; - } - this.loginContextName = loginContextName; - this.useTicketCache = useTicketCache; - this.keytabFile = keytabFile; - this.principal = principal; - LOG.info("JaasConfiguration loginContextName=" + loginContextName + - " principal=" + principal + " useTicketCache=" + useTicketCache + - " keytabFile=" + keytabFile); - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String appName) { - if (loginContextName.equals(appName)) { - if (!useTicketCache) { - KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile); - KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true"); - } - KEYTAB_KERBEROS_OPTIONS.put("principal", principal); - KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false"); - return KEYTAB_KERBEROS_CONF; - } - if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName); - return(null); - } - } - - // - // Helper methods - // - - /** - * Join the prefix znode name with the suffix znode name to generate a proper - * full znode name. - * - * Assumes prefix does not end with slash and suffix does not begin with it. - * - * @param prefix beginning of znode name - * @param suffix ending of znode name - * @return result of properly joining prefix with suffix - */ - public static String joinZNode(String prefix, String suffix) { - return prefix + ZNODE_PATH_SEPARATOR + suffix; - } - - /** - * Returns the full path of the immediate parent of the specified node. - * @param node path to get parent of - * @return parent of path, null if passed the root node or an invalid node - */ - public static String getParent(String node) { - int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR); - return idx <= 0 ? null : node.substring(0, idx); - } - - /** - * Get the name of the current node from the specified fully-qualified path. - * @param path fully-qualified path - * @return name of the current node - */ - public static String getNodeName(String path) { - return path.substring(path.lastIndexOf("/")+1); - } - - // - // Existence checks and watches - // - - /** - * Watch the specified znode for delete/create/change events. The watcher is - * set whether or not the node exists. If the node already exists, the method - * returns true. If the node does not exist, the method returns false. - * - * @param zkw zk reference - * @param znode path of node to watch - * @return true if znode exists, false if does not exist or error - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw); - boolean exists = s != null ? true : false; - if (exists) { - LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode)); - } else { - LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode)); - } - return exists; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); - zkw.keeperException(e); - return false; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); - zkw.interruptedException(e); - return false; - } - } - - /** - * Watch the specified znode, but only if exists. Useful when watching - * for deletions. Uses .getData() (and handles NoNodeException) instead - * of .exists() to accomplish this, as .getData() will only set a watch if - * the znode exists. - * @param zkw zk reference - * @param znode path of node to watch - * @return true if the watch is set, false if node does not exists - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - zkw.getRecoverableZooKeeper().getData(znode, true, null); - return true; - } catch (NoNodeException e) { - return false; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); - zkw.interruptedException(e); - return false; - } - } - - /** - * Check if the specified node exists. Sets no watches. - * - * @param zkw zk reference - * @param znode path of node to watch - * @return version of the node if it exists, -1 if does not exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static int checkExists(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - Stat s = zkw.getRecoverableZooKeeper().exists(znode, null); - return s != null ? s.getVersion() : -1; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); - zkw.keeperException(e); - return -1; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); - zkw.interruptedException(e); - return -1; - } - } - - // - // Znode listings - // - - /** - * Lists the children znodes of the specified znode. Also sets a watch on - * the specified znode which will capture a NodeDeleted event on the specified - * znode as well as NodeChildrenChanged if any children of the specified znode - * are created or deleted. - * - * Returns null if the specified node does not exist. Otherwise returns a - * list of children of the specified node. If the node exists but it has no - * children, an empty list will be returned. - * - * @param zkw zk reference - * @param znode path of node to list and watch children of - * @return list of children of the specified node, an empty list if the node - * exists but has no children, and null if the node does not exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static List listChildrenAndWatchForNewChildren( - ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - List children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw); - return children; - } catch(KeeperException.NoNodeException ke) { - LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + - "because node does not exist (not an error)")); - return null; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); - zkw.keeperException(e); - return null; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); - zkw.interruptedException(e); - return null; - } - } - - /** - * List all the children of the specified znode, setting a watch for children - * changes and also setting a watch on every individual child in order to get - * the NodeCreated and NodeDeleted events. - * @param zkw zookeeper reference - * @param znode node to get children of and watch - * @return list of znode names, null if the node doesn't exist - * @throws KeeperException - */ - public static List listChildrenAndWatchThem(ZooKeeperWatcher zkw, - String znode) throws KeeperException { - List children = listChildrenAndWatchForNewChildren(zkw, znode); - if (children == null) { - return null; - } - for (String child : children) { - watchAndCheckExists(zkw, joinZNode(znode, child)); - } - return children; - } - - /** - * Lists the children of the specified znode without setting any watches. - * - * Sets no watches at all, this method is best effort. - * - * Returns an empty list if the node has no children. Returns null if the - * parent node itself does not exist. - * - * @param zkw zookeeper reference - * @param znode node to get children - * @return list of data of children of specified znode, empty if no children, - * null if parent does not exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static List listChildrenNoWatch(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - List children = null; - try { - // List the children without watching - children = zkw.getRecoverableZooKeeper().getChildren(znode, null); - } catch(KeeperException.NoNodeException nne) { - return null; - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - } - return children; - } - - /** - * Simple class to hold a node path and node data. - * @deprecated Unused - */ - @Deprecated - public static class NodeAndData { - private String node; - private byte [] data; - public NodeAndData(String node, byte [] data) { - this.node = node; - this.data = data; - } - public String getNode() { - return node; - } - public byte [] getData() { - return data; - } - @Override - public String toString() { - return node; - } - public boolean isEmpty() { - return (data == null || data.length == 0); - } - } - - /** - * Checks if the specified znode has any children. Sets no watches. - * - * Returns true if the node exists and has children. Returns false if the - * node does not exist or if the node does not have any children. - * - * Used during master initialization to determine if the master is a - * failed-over-to master or the first master during initial cluster startup. - * If the directory for regionserver ephemeral nodes is empty then this is - * a cluster startup, if not then it is not cluster startup. - * - * @param zkw zk reference - * @param znode path of node to check for children of - * @return true if node has children, false if not or node does not exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty(); - } catch(KeeperException.NoNodeException ke) { - LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + - "because node does not exist (not an error)")); - return false; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); - zkw.keeperException(e); - return false; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); - zkw.interruptedException(e); - return false; - } - } - - /** - * Get the number of children of the specified node. - * - * If the node does not exist or has no children, returns 0. - * - * Sets no watches at all. - * - * @param zkw zk reference - * @param znode path of node to count children of - * @return number of children of specified node, 0 if none or parent does not - * exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - try { - Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null); - return stat == null ? 0 : stat.getNumChildren(); - } catch(KeeperException e) { - LOG.warn(zkw.prefix("Unable to get children of node " + znode)); - zkw.keeperException(e); - } catch(InterruptedException e) { - zkw.interruptedException(e); - } - return 0; - } - - // - // Data retrieval - // - - /** - * Get znode data. Does not set a watcher. - * @return ZNode data, null if the node does not exist or if there is an - * error. - */ - public static byte [] getData(ZooKeeperWatcher zkw, String znode) - throws KeeperException, InterruptedException { - try { - byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null); - logRetrievedMsg(zkw, znode, data, false); - return data; - } catch (KeeperException.NoNodeException e) { - LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not an error)")); - return null; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); - zkw.keeperException(e); - return null; - } - } - - /** - * Get the data at the specified znode and set a watch. - * - * Returns the data and sets a watch if the node exists. Returns null and no - * watch is set if the node does not exist or there is an exception. - * - * @param zkw zk reference - * @param znode path of node - * @return data of the specified znode, or null - * @throws KeeperException if unexpected zookeeper exception - */ - public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - return getDataInternal(zkw, znode, null, true); - } - - /** - * Get the data at the specified znode and set a watch. - * - * Returns the data and sets a watch if the node exists. Returns null and no - * watch is set if the node does not exist or there is an exception. - * - * @param zkw zk reference - * @param znode path of node - * @param stat object to populate the version of the znode - * @return data of the specified znode, or null - * @throws KeeperException if unexpected zookeeper exception - */ - public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode, - Stat stat) throws KeeperException { - return getDataInternal(zkw, znode, stat, true); - } - - private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat, - boolean watcherSet) - throws KeeperException { - try { - byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat); - logRetrievedMsg(zkw, znode, data, watcherSet); - return data; - } catch (KeeperException.NoNodeException e) { - // This log can get pretty annoying when we cycle on 100ms waits. - // Enable trace if you really want to see it. - LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not an error)")); - return null; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); - zkw.keeperException(e); - return null; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); - zkw.interruptedException(e); - return null; - } - } - - /** - * Get the data at the specified znode without setting a watch. - * - * Returns the data if the node exists. Returns null if the node does not - * exist. - * - * Sets the stats of the node in the passed Stat object. Pass a null stat if - * not interested. - * - * @param zkw zk reference - * @param znode path of node - * @param stat node status to get if node exists - * @return data of the specified znode, or null if node does not exist - * @throws KeeperException if unexpected zookeeper exception - */ - public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode, - Stat stat) - throws KeeperException { - try { - byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat); - logRetrievedMsg(zkw, znode, data, false); - return data; - } catch (KeeperException.NoNodeException e) { - LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not necessarily an error)")); - return null; - } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); - zkw.keeperException(e); - return null; - } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); - zkw.interruptedException(e); - return null; - } - } - - /** - * Returns the date of child znodes of the specified znode. Also sets a watch on - * the specified znode which will capture a NodeDeleted event on the specified - * znode as well as NodeChildrenChanged if any children of the specified znode - * are created or deleted. - * - * Returns null if the specified node does not exist. Otherwise returns a - * list of children of the specified node. If the node exists but it has no - * children, an empty list will be returned. - * - * @param zkw zk reference - * @param baseNode path of node to list and watch children of - * @return list of data of children of the specified node, an empty list if the node - * exists but has no children, and null if the node does not exist - * @throws KeeperException if unexpected zookeeper exception - * @deprecated Unused - */ - @Deprecated - public static List getChildDataAndWatchForNewChildren( - ZooKeeperWatcher zkw, String baseNode) throws KeeperException { - List nodes = - ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); - if (nodes != null) { - List newNodes = new ArrayList<>(); - for (String node : nodes) { - String nodePath = ZKUtil.joinZNode(baseNode, node); - byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath); - newNodes.add(new NodeAndData(nodePath, data)); - } - return newNodes; - } - return null; - } - - /** - * Update the data of an existing node with the expected version to have the - * specified data. - * - * Throws an exception if there is a version mismatch or some other problem. - * - * Sets no watches under any conditions. - * - * @param zkw zk reference - * @param znode - * @param data - * @param expectedVersion - * @throws KeeperException if unexpected zookeeper exception - * @throws KeeperException.BadVersionException if version mismatch - * @deprecated Unused - */ - @Deprecated - public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode, - byte [] data, int expectedVersion) - throws KeeperException { - try { - zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion); - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - } - } - - // - // Data setting - // - - /** - * Sets the data of the existing znode to be the specified data. Ensures that - * the current data has the specified expected version. - * - *

If the node does not exist, a {@link NoNodeException} will be thrown. - * - *

If their is a version mismatch, method returns null. - * - *

No watches are set but setting data will trigger other watchers of this - * node. - * - *

If there is another problem, a KeeperException will be thrown. - * - * @param zkw zk reference - * @param znode path of node - * @param data data to set for node - * @param expectedVersion version expected when setting data - * @return true if data set, false if version mismatch - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean setData(ZooKeeperWatcher zkw, String znode, - byte [] data, int expectedVersion) - throws KeeperException, KeeperException.NoNodeException { - try { - return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null; - } catch (InterruptedException e) { - zkw.interruptedException(e); - return false; - } - } - - /** - * Set data into node creating node if it doesn't yet exist. - * Does not set watch. - * - * @param zkw zk reference - * @param znode path of node - * @param data data to set for node - * @throws KeeperException - */ - public static void createSetData(final ZooKeeperWatcher zkw, final String znode, - final byte [] data) - throws KeeperException { - if (checkExists(zkw, znode) == -1) { - ZKUtil.createWithParents(zkw, znode, data); - } else { - ZKUtil.setData(zkw, znode, data); - } - } - - /** - * Sets the data of the existing znode to be the specified data. The node - * must exist but no checks are done on the existing data or version. - * - *

If the node does not exist, a {@link NoNodeException} will be thrown. - * - *

No watches are set but setting data will trigger other watchers of this - * node. - * - *

If there is another problem, a KeeperException will be thrown. - * - * @param zkw zk reference - * @param znode path of node - * @param data data to set for node - * @throws KeeperException if unexpected zookeeper exception - */ - public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data) - throws KeeperException, KeeperException.NoNodeException { - setData(zkw, (SetData)ZKUtilOp.setData(znode, data)); - } - - private static void setData(ZooKeeperWatcher zkw, SetData setData) - throws KeeperException, KeeperException.NoNodeException { - SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord(); - setData(zkw, sd.getPath(), sd.getData(), sd.getVersion()); - } - - /** - * Returns whether or not secure authentication is enabled - * (whether hbase.security.authentication is set to - * kerberos. - */ - public static boolean isSecureZooKeeper(Configuration conf) { - // Detection for embedded HBase client with jaas configuration - // defined for third party programs. - try { - javax.security.auth.login.Configuration testConfig = - javax.security.auth.login.Configuration.getConfiguration(); - if (testConfig.getAppConfigurationEntry("Client") == null - && testConfig.getAppConfigurationEntry( - JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null - && testConfig.getAppConfigurationEntry( - JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null - && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null - && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) { - - return false; - } - } catch(Exception e) { - // No Jaas configuration defined. - return false; - } - - // Master & RSs uses hbase.zookeeper.client.* - return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication")); - } - - private static ArrayList createACL(ZooKeeperWatcher zkw, String node) { - return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration())); - } - - public static ArrayList createACL(ZooKeeperWatcher zkw, String node, - boolean isSecureZooKeeper) { - if (!node.startsWith(zkw.znodePaths.baseZNode)) { - return Ids.OPEN_ACL_UNSAFE; - } - if (isSecureZooKeeper) { - ArrayList acls = new ArrayList<>(); - // add permission to hbase supper user - String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY); - String hbaseUser = null; - try { - hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - LOG.debug("Could not acquire current User.", e); - } - if (superUsers != null) { - List groups = new ArrayList<>(); - for (String user : superUsers) { - if (AuthUtil.isGroupPrincipal(user)) { - // TODO: Set node ACL for groups when ZK supports this feature - groups.add(user); - } else { - if(!user.equals(hbaseUser)) { - acls.add(new ACL(Perms.ALL, new Id("sasl", user))); - } - } - } - if (!groups.isEmpty()) { - LOG.warn("Znode ACL setting for group " + groups - + " is skipped, ZooKeeper doesn't support this feature presently."); - } - } - // Certain znodes are accessed directly by the client, - // so they must be readable by non-authenticated clients - if (zkw.isClientReadable(node)) { - acls.addAll(Ids.CREATOR_ALL_ACL); - acls.addAll(Ids.READ_ACL_UNSAFE); - } else { - acls.addAll(Ids.CREATOR_ALL_ACL); - } - return acls; - } else { - return Ids.OPEN_ACL_UNSAFE; - } - } - - // - // Node creation - // - - /** - * - * Set the specified znode to be an ephemeral node carrying the specified - * data. - * - * If the node is created successfully, a watcher is also set on the node. - * - * If the node is not created successfully because it already exists, this - * method will also set a watcher on the node. - * - * If there is another problem, a KeeperException will be thrown. - * - * @param zkw zk reference - * @param znode path of node - * @param data data of node - * @return true if node created, false if not, watch set in both cases - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw, - String znode, byte [] data) - throws KeeperException { - boolean ret = true; - try { - zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), - CreateMode.EPHEMERAL); - } catch (KeeperException.NodeExistsException nee) { - ret = false; - } catch (InterruptedException e) { - LOG.info("Interrupted", e); - Thread.currentThread().interrupt(); - } - if(!watchAndCheckExists(zkw, znode)) { - // It did exist but now it doesn't, try again - return createEphemeralNodeAndWatch(zkw, znode, data); - } - return ret; - } - - /** - * Creates the specified znode to be a persistent node carrying the specified - * data. - * - * Returns true if the node was successfully created, false if the node - * already existed. - * - * If the node is created successfully, a watcher is also set on the node. - * - * If the node is not created successfully because it already exists, this - * method will also set a watcher on the node but return false. - * - * If there is another problem, a KeeperException will be thrown. - * - * @param zkw zk reference - * @param znode path of node - * @param data data of node - * @return true if node created, false if not, watch set in both cases - * @throws KeeperException if unexpected zookeeper exception - */ - public static boolean createNodeIfNotExistsAndWatch( - ZooKeeperWatcher zkw, String znode, byte [] data) - throws KeeperException { - boolean ret = true; - try { - zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), - CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException nee) { - ret = false; - } catch (InterruptedException e) { - zkw.interruptedException(e); - return false; - } - try { - zkw.getRecoverableZooKeeper().exists(znode, zkw); - } catch (InterruptedException e) { - zkw.interruptedException(e); - return false; - } - return ret; - } - - /** - * Creates the specified znode with the specified data but does not watch it. - * - * Returns the znode of the newly created node - * - * If there is another problem, a KeeperException will be thrown. - * - * @param zkw zk reference - * @param znode path of node - * @param data data of node - * @param createMode specifying whether the node to be created is ephemeral and/or sequential - * @return true name of the newly created znode or null - * @throws KeeperException if unexpected zookeeper exception - */ - public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode, - byte[] data, CreateMode createMode) throws KeeperException { - - String createdZNode = null; - try { - createdZNode = zkw.getRecoverableZooKeeper().create(znode, data, - createACL(zkw, znode), createMode); - } catch (KeeperException.NodeExistsException nee) { - return znode; - } catch (InterruptedException e) { - zkw.interruptedException(e); - return null; - } - return createdZNode; - } - - /** - * Creates the specified node with the specified data and watches it. - * - *

Throws an exception if the node already exists. - * - *

The node created is persistent and open access. - * - *

Returns the version number of the created node if successful. - * - * @param zkw zk reference - * @param znode path of node to create - * @param data data of node to create - * @return version of node created - * @throws KeeperException if unexpected zookeeper exception - * @throws KeeperException.NodeExistsException if node already exists - */ - public static int createAndWatch(ZooKeeperWatcher zkw, - String znode, byte [] data) - throws KeeperException, KeeperException.NodeExistsException { - try { - zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), - CreateMode.PERSISTENT); - Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw); - if (stat == null){ - // Likely a race condition. Someone deleted the znode. - throw KeeperException.create(KeeperException.Code.SYSTEMERROR, - "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode); - } - return stat.getVersion(); - } catch (InterruptedException e) { - zkw.interruptedException(e); - return -1; - } - } - - /** - * Async creates the specified node with the specified data. - * - *

Throws an exception if the node already exists. - * - *

The node created is persistent and open access. - * - * @param zkw zk reference - * @param znode path of node to create - * @param data data of node to create - * @param cb - * @param ctx - */ - public static void asyncCreate(ZooKeeperWatcher zkw, - String znode, byte [] data, final AsyncCallback.StringCallback cb, - final Object ctx) { - zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data, - createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx); - } - - /** - * Creates the specified node, iff the node does not exist. Does not set a - * watch and fails silently if the node already exists. - * - * The node created is persistent and open access. - * - * @param zkw zk reference - * @param znode path of node - * @throws KeeperException if unexpected zookeeper exception - */ - public static void createAndFailSilent(ZooKeeperWatcher zkw, - String znode) throws KeeperException { - createAndFailSilent(zkw, znode, new byte[0]); - } - - /** - * Creates the specified node containing specified data, iff the node does not exist. Does - * not set a watch and fails silently if the node already exists. - * - * The node created is persistent and open access. - * - * @param zkw zk reference - * @param znode path of node - * @param data a byte array data to store in the znode - * @throws KeeperException if unexpected zookeeper exception - */ - public static void createAndFailSilent(ZooKeeperWatcher zkw, - String znode, byte[] data) - throws KeeperException { - createAndFailSilent(zkw, - (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data)); - } - - private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs) - throws KeeperException { - CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord(); - String znode = create.getPath(); - try { - RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper(); - if (zk.exists(znode, false) == null) { - zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags())); - } - } catch(KeeperException.NodeExistsException nee) { - } catch(KeeperException.NoAuthException nee){ - try { - if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) { - // If we failed to create the file and it does not already exist. - throw(nee); - } - } catch (InterruptedException ie) { - zkw.interruptedException(ie); - } - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - } - } - - /** - * Creates the specified node and all parent nodes required for it to exist. - * - * No watches are set and no errors are thrown if the node already exists. - * - * The nodes created are persistent and open access. - * - * @param zkw zk reference - * @param znode path of node - * @throws KeeperException if unexpected zookeeper exception - */ - public static void createWithParents(ZooKeeperWatcher zkw, String znode) - throws KeeperException { - createWithParents(zkw, znode, new byte[0]); - } - - /** - * Creates the specified node and all parent nodes required for it to exist. The creation of - * parent znodes is not atomic with the leafe znode creation but the data is written atomically - * when the leaf node is created. - * - * No watches are set and no errors are thrown if the node already exists. - * - * The nodes created are persistent and open access. - * - * @param zkw zk reference - * @param znode path of node - * @throws KeeperException if unexpected zookeeper exception - */ - public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data) - throws KeeperException { - try { - if(znode == null) { - return; - } - zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), - CreateMode.PERSISTENT); - } catch(KeeperException.NodeExistsException nee) { - return; - } catch(KeeperException.NoNodeException nne) { - createWithParents(zkw, getParent(znode)); - createWithParents(zkw, znode, data); - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - } - } - - // - // Deletes - // - - /** - * Delete the specified node. Sets no watches. Throws all exceptions. - */ - public static void deleteNode(ZooKeeperWatcher zkw, String node) - throws KeeperException { - deleteNode(zkw, node, -1); - } - - /** - * Delete the specified node with the specified version. Sets no watches. - * Throws all exceptions. - */ - public static boolean deleteNode(ZooKeeperWatcher zkw, String node, - int version) - throws KeeperException { - try { - zkw.getRecoverableZooKeeper().delete(node, version); - return true; - } catch(KeeperException.BadVersionException bve) { - return false; - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - return false; - } - } - - /** - * Deletes the specified node. Fails silent if the node does not exist. - * @param zkw - * @param node - * @throws KeeperException - */ - public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node) - throws KeeperException { - deleteNodeFailSilent(zkw, - (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node)); - } - - private static void deleteNodeFailSilent(ZooKeeperWatcher zkw, - DeleteNodeFailSilent dnfs) throws KeeperException { - DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord(); - try { - zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion()); - } catch(KeeperException.NoNodeException nne) { - } catch(InterruptedException ie) { - zkw.interruptedException(ie); - } - } - - - /** - * Delete the specified node and all of it's children. - *

- * If the node does not exist, just returns. - *

- * Sets no watches. Throws all exceptions besides dealing with deletion of - * children. - */ - public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node) - throws KeeperException { - deleteNodeRecursivelyMultiOrSequential(zkw, true, node); - } - - /** - * Delete all the children of the specified node but not the node itself. - * - * Sets no watches. Throws all exceptions besides dealing with deletion of - * children. - * - * @throws KeeperException - */ - public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node) - throws KeeperException { - deleteChildrenRecursivelyMultiOrSequential(zkw, true, node); - } - - /** - * Delete all the children of the specified node but not the node itself. This - * will first traverse the znode tree for listing the children and then delete - * these znodes using multi-update api or sequential based on the specified - * configurations. - *

- * Sets no watches. Throws all exceptions besides dealing with deletion of - * children. - *

- * If the following is true: - *

    - *
  • runSequentialOnMultiFailure is true - *
- * on calling multi, we get a ZooKeeper exception that can be handled by a - * sequential call(*), we retry the operations one-by-one (sequentially). - * - * @param zkw - * - zk reference - * @param runSequentialOnMultiFailure - * - if true when we get a ZooKeeper exception that could retry the - * operations one-by-one (sequentially) - * @param pathRoots - * - path of the parent node(s) - * @throws KeeperException.NotEmptyException - * if node has children while deleting - * @throws KeeperException - * if unexpected ZooKeeper exception - * @throws IllegalArgumentException - * if an invalid path is specified - */ - public static void deleteChildrenRecursivelyMultiOrSequential( - ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure, - String... pathRoots) throws KeeperException { - if (pathRoots == null || pathRoots.length <= 0) { - LOG.warn("Given path is not valid!"); - return; - } - List ops = new ArrayList<>(); - for (String eachRoot : pathRoots) { - List children = listChildrenBFSNoWatch(zkw, eachRoot); - // Delete the leaves first and eventually get rid of the root - for (int i = children.size() - 1; i >= 0; --i) { - ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i))); - } - } - // atleast one element should exist - if (ops.size() > 0) { - multiOrSequential(zkw, ops, runSequentialOnMultiFailure); - } - } - - /** - * Delete the specified node and its children. This traverse the - * znode tree for listing the children and then delete - * these znodes including the parent using multi-update api or - * sequential based on the specified configurations. - *

- * Sets no watches. Throws all exceptions besides dealing with deletion of - * children. - *

- * If the following is true: - *

    - *
  • runSequentialOnMultiFailure is true - *
- * on calling multi, we get a ZooKeeper exception that can be handled by a - * sequential call(*), we retry the operations one-by-one (sequentially). - * - * @param zkw - * - zk reference - * @param runSequentialOnMultiFailure - * - if true when we get a ZooKeeper exception that could retry the - * operations one-by-one (sequentially) - * @param pathRoots - * - path of the parent node(s) - * @throws KeeperException.NotEmptyException - * if node has children while deleting - * @throws KeeperException - * if unexpected ZooKeeper exception - * @throws IllegalArgumentException - * if an invalid path is specified - */ - public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw, - boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException { - if (pathRoots == null || pathRoots.length <= 0) { - LOG.warn("Given path is not valid!"); - return; - } - List ops = new ArrayList<>(); - for (String eachRoot : pathRoots) { - // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted - // recursively, must set another watch, get notified of delete node - List children = listChildrenBFSAndWatchThem(zkw, eachRoot); - // Delete the leaves first and eventually get rid of the root - for (int i = children.size() - 1; i >= 0; --i) { - ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i))); - } - try { - if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) { - ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot)); - } - } catch (InterruptedException e) { - zkw.interruptedException(e); - } - } - // atleast one element should exist - if (ops.size() > 0) { - multiOrSequential(zkw, ops, runSequentialOnMultiFailure); - } - } - - /** - * BFS Traversal of all the children under path, with the entries in the list, - * in the same order as that of the traversal. Lists all the children without - * setting any watches. - * - * @param zkw - * - zk reference - * @param znode - * - path of node - * @return list of children znodes under the path - * @throws KeeperException - * if unexpected ZooKeeper exception - */ - private static List listChildrenBFSNoWatch(ZooKeeperWatcher zkw, - final String znode) throws KeeperException { - Deque queue = new LinkedList<>(); - List tree = new ArrayList<>(); - queue.add(znode); - while (true) { - String node = queue.pollFirst(); - if (node == null) { - break; - } - List children = listChildrenNoWatch(zkw, node); - if (children == null) { - continue; - } - for (final String child : children) { - final String childPath = node + "/" + child; - queue.add(childPath); - tree.add(childPath); - } - } - return tree; - } - - /** - * BFS Traversal of all the children under path, with the entries in the list, - * in the same order as that of the traversal. - * Lists all the children and set watches on to them. - * - * @param zkw - * - zk reference - * @param znode - * - path of node - * @return list of children znodes under the path - * @throws KeeperException - * if unexpected ZooKeeper exception - */ - private static List listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode) - throws KeeperException { - Deque queue = new LinkedList<>(); - List tree = new ArrayList<>(); - queue.add(znode); - while (true) { - String node = queue.pollFirst(); - if (node == null) { - break; - } - List children = listChildrenAndWatchThem(zkw, node); - if (children == null) { - continue; - } - for (final String child : children) { - final String childPath = node + "/" + child; - queue.add(childPath); - tree.add(childPath); - } - } - return tree; - } - - /** - * Represents an action taken by ZKUtil, e.g. createAndFailSilent. - * These actions are higher-level than ZKOp actions, which represent - * individual actions in the ZooKeeper API, like create. - */ - public abstract static class ZKUtilOp { - private String path; - - private ZKUtilOp(String path) { - this.path = path; - } - - /** - * @return a createAndFailSilent ZKUtilOp - */ - public static ZKUtilOp createAndFailSilent(String path, byte[] data) { - return new CreateAndFailSilent(path, data); - } - - /** - * @return a deleteNodeFailSilent ZKUtilOP - */ - public static ZKUtilOp deleteNodeFailSilent(String path) { - return new DeleteNodeFailSilent(path); - } - - /** - * @return a setData ZKUtilOp - */ - public static ZKUtilOp setData(String path, byte [] data) { - return new SetData(path, data); - } - - /** - * @return path to znode where the ZKOp will occur - */ - public String getPath() { - return path; - } - - /** - * ZKUtilOp representing createAndFailSilent in ZooKeeper - * (attempt to create node, ignore error if already exists) - */ - public static class CreateAndFailSilent extends ZKUtilOp { - private byte [] data; - - private CreateAndFailSilent(String path, byte [] data) { - super(path); - this.data = data; - } - - public byte[] getData() { - return data; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof CreateAndFailSilent)) return false; - - CreateAndFailSilent op = (CreateAndFailSilent) o; - return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); - } - - @Override - public int hashCode() { - int ret = 17 + getPath().hashCode() * 31; - return ret * 31 + Bytes.hashCode(data); - } - } - - /** - * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper - * (attempt to delete node, ignore error if node doesn't exist) - */ - public static class DeleteNodeFailSilent extends ZKUtilOp { - private DeleteNodeFailSilent(String path) { - super(path); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof DeleteNodeFailSilent)) return false; - - return super.equals(o); - } - - @Override - public int hashCode() { - return getPath().hashCode(); - } - } - - /** - * ZKUtilOp representing setData in ZooKeeper - */ - public static class SetData extends ZKUtilOp { - private byte [] data; - - private SetData(String path, byte [] data) { - super(path); - this.data = data; - } - - public byte[] getData() { - return data; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof SetData)) return false; - - SetData op = (SetData) o; - return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); - } - - @Override - public int hashCode() { - int ret = getPath().hashCode(); - return ret * 31 + Bytes.hashCode(data); - } - } - } - - /** - * Convert from ZKUtilOp to ZKOp - */ - private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op) - throws UnsupportedOperationException { - if(op == null) return null; - - if (op instanceof CreateAndFailSilent) { - CreateAndFailSilent cafs = (CreateAndFailSilent)op; - return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()), - CreateMode.PERSISTENT); - } else if (op instanceof DeleteNodeFailSilent) { - DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op; - return Op.delete(dnfs.getPath(), -1); - } else if (op instanceof SetData) { - SetData sd = (SetData)op; - return Op.setData(sd.getPath(), sd.getData(), -1); - } else { - throw new UnsupportedOperationException("Unexpected ZKUtilOp type: " - + op.getClass().getName()); - } - } - - /** - * Use ZooKeeper's multi-update functionality. - * - * If all of the following are true: - * - runSequentialOnMultiFailure is true - * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*) - * Then: - * - we retry the operations one-by-one (sequentially) - * - * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi, - * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who - * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught). - * This will cause all operations in the multi to fail, however, because - * the NodeExistsException that zk.create throws will fail the multi transaction. - * In this case, if the previous conditions hold, the commands are run sequentially, which should - * result in the correct final state, but means that the operations will not run atomically. - * - * @throws KeeperException - */ - public static void multiOrSequential(ZooKeeperWatcher zkw, List ops, - boolean runSequentialOnMultiFailure) throws KeeperException { - if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) { - LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always."); - } - if (ops == null) return; - - List zkOps = new LinkedList<>(); - for (ZKUtilOp op : ops) { - zkOps.add(toZooKeeperOp(zkw, op)); - } - try { - zkw.getRecoverableZooKeeper().multi(zkOps); - } catch (KeeperException ke) { - switch (ke.code()) { - case NODEEXISTS: - case NONODE: - case BADVERSION: - case NOAUTH: - // if we get an exception that could be solved by running sequentially - // (and the client asked us to), then break out and run sequentially - if (runSequentialOnMultiFailure) { - LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "." - + " Attempting to run operations sequentially because" - + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + "."); - processSequentially(zkw, ops); - break; - } - default: - throw ke; - } - } catch (InterruptedException ie) { - zkw.interruptedException(ie); - } - } - - private static void processSequentially(ZooKeeperWatcher zkw, List ops) - throws KeeperException, NoNodeException { - for (ZKUtilOp op : ops) { - if (op instanceof CreateAndFailSilent) { - createAndFailSilent(zkw, (CreateAndFailSilent) op); - } else if (op instanceof DeleteNodeFailSilent) { - deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op); - } else if (op instanceof SetData) { - setData(zkw, (SetData) op); - } else { - throw new UnsupportedOperationException("Unexpected ZKUtilOp type: " - + op.getClass().getName()); - } - } - } + public static final int DEFAULT_ZOOKEEPER_DUMP_CONNECTION_TIMEOUT = 1000; // // ZooKeeper cluster information // - /** @return String dump of everything in ZooKeeper. */ public static String dump(ZooKeeperWatcher zkw) { StringBuilder sb = new StringBuilder(); @@ -1716,7 +68,7 @@ public class ZKUtil { sb.append("<>"); } sb.append("\nBackup master addresses:"); - for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) { + for (String child : ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) { sb.append("\n ").append(child); } sb.append("\nRegion server holding hbase:meta: " @@ -1729,7 +81,7 @@ public class ZKUtil { + new MetaTableLocator().getMetaRegionLocation(zkw, i)); } sb.append("\nRegion servers:"); - for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) { + for (String child : ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) { sb.append("\n ").append(child); } try { @@ -1742,7 +94,9 @@ public class ZKUtil { for (String server : servers) { sb.append("\n ").append(server); try { - String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut); + int zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout", + DEFAULT_ZOOKEEPER_DUMP_CONNECTION_TIMEOUT); + String[] stat = ZooKeeperUtil.getServerStats(server, zkDumpConnectionTimeOut); if (stat == null) { sb.append("[Error] invalid quorum server: " + server); @@ -1772,35 +126,18 @@ public class ZKUtil { private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb) throws KeeperException { String replicationZnode = zkw.znodePaths.replicationZNode; - if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return; + if (ZooKeeperUtil.checkExists(zkw, replicationZnode) == -1) return; // do a ls -r on this znode sb.append("\n").append(replicationZnode).append(": "); - List children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode); + List children = ZooKeeperUtil.listChildrenNoWatch(zkw, replicationZnode); for (String child : children) { - String znode = joinZNode(replicationZnode, child); + String znode = ZooKeeperUtil.joinZNode(replicationZnode, child); if (znode.equals(zkw.znodePaths.peersZNode)) { appendPeersZnodes(zkw, znode, sb); } else if (znode.equals(zkw.znodePaths.queuesZNode)) { appendRSZnodes(zkw, znode, sb); } else if (znode.equals(zkw.znodePaths.hfileRefsZNode)) { - appendHFileRefsZnodes(zkw, znode, sb); - } - } - } - - private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode, - StringBuilder sb) throws KeeperException { - sb.append("\n").append(hfileRefsZnode).append(": "); - for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) { - String znodeToProcess = ZKUtil.joinZNode(hfileRefsZnode, peerIdZnode); - sb.append("\n").append(znodeToProcess).append(": "); - List peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess); - int size = peerHFileRefsZnodes.size(); - for (int i = 0; i < size; i++) { - sb.append(peerHFileRefsZnodes.get(i)); - if (i != size - 1) { - sb.append(", "); - } + ZooKeeperUtil.appendHFileRefsZnodes(zkw, znode, sb); } } } @@ -1825,7 +162,7 @@ public class ZKUtil { sb.append("\n").append(znodeToProcess).append(": "); byte[] data; try { - data = ZKUtil.getData(zkw, znodeToProcess); + data = ZooKeeperUtil.getData(zkw, znodeToProcess); } catch (InterruptedException e) { zkw.interruptedException(e); return; @@ -1833,7 +170,7 @@ public class ZKUtil { if (data != null && data.length > 0) { // log position long position = 0; try { - position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess)); + position = ZKUtil.parseWALPositionFrom(ZooKeeperUtil.getData(zkw, znodeToProcess)); sb.append(position); } catch (DeserializationException ignored) { } catch (InterruptedException e) { @@ -1841,8 +178,8 @@ public class ZKUtil { return; } } - for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { - stack.add(ZKUtil.joinZNode(znodeToProcess, zNodeChild)); + for (String zNodeChild : ZooKeeperUtil.listChildrenNoWatch(zkw, znodeToProcess)) { + stack.add(ZooKeeperUtil.joinZNode(znodeToProcess, zNodeChild)); } } while (stack.size() > 0); } @@ -1851,11 +188,11 @@ public class ZKUtil { StringBuilder sb) throws KeeperException { int pblen = ProtobufUtil.lengthOfPBMagic(); sb.append("\n").append(peersZnode).append(": "); - for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) { - String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode); + for (String peerIdZnode : ZooKeeperUtil.listChildrenNoWatch(zkw, peersZnode)) { + String znodeToProcess = ZooKeeperUtil.joinZNode(peersZnode, peerIdZnode); byte[] data; try { - data = ZKUtil.getData(zkw, znodeToProcess); + data = ZooKeeperUtil.getData(zkw, znodeToProcess); } catch (InterruptedException e) { zkw.interruptedException(e); return; @@ -1880,13 +217,13 @@ public class ZKUtil { String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state", "peer-state"); int pblen = ProtobufUtil.lengthOfPBMagic(); - for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { + for (String child : ZooKeeperUtil.listChildrenNoWatch(zkw, znodeToProcess)) { if (!child.equals(peerState)) continue; - String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child); + String peerStateZnode = ZooKeeperUtil.joinZNode(znodeToProcess, child); sb.append("\n").append(peerStateZnode).append(": "); byte[] peerStateData; try { - peerStateData = ZKUtil.getData(zkw, peerStateZnode); + peerStateData = ZooKeeperUtil.getData(zkw, peerStateZnode); ReplicationProtos.ReplicationState.Builder builder = ReplicationProtos.ReplicationState.newBuilder(); ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen); @@ -1900,156 +237,6 @@ public class ZKUtil { } } - /** - * Gets the statistics from the given server. - * - * @param server The server to get the statistics from. - * @param timeout The socket timeout to use. - * @return The array of response strings. - * @throws IOException When the socket communication fails. - */ - public static String[] getServerStats(String server, int timeout) - throws IOException { - String[] sp = server.split(":"); - if (sp == null || sp.length == 0) { - return null; - } - - String host = sp[0]; - int port = sp.length > 1 ? Integer.parseInt(sp[1]) - : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT; - - InetSocketAddress sockAddr = new InetSocketAddress(host, port); - try (Socket socket = new Socket()) { - socket.connect(sockAddr, timeout); - - socket.setSoTimeout(timeout); - try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true); - BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { - out.println("stat"); - out.flush(); - ArrayList res = new ArrayList<>(); - while (true) { - String line = in.readLine(); - if (line != null) { - res.add(line); - } else { - break; - } - } - return res.toArray(new String[res.size()]); - } - } - } - - private static void logRetrievedMsg(final ZooKeeperWatcher zkw, - final String znode, final byte [] data, final boolean watcherSet) { - if (!LOG.isTraceEnabled()) return; - LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) + - " byte(s) of data from znode " + znode + - (watcherSet? " and set watcher; ": "; data=") + - (data == null? "null": data.length == 0? "empty": ( - znode.startsWith(zkw.znodePaths.metaZNodePrefix)? - getServerNameOrEmptyString(data): - znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)? - getServerNameOrEmptyString(data): - StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); - } - - private static String getServerNameOrEmptyString(final byte [] data) { - try { - return ProtobufUtil.parseServerNameFrom(data).toString(); - } catch (DeserializationException e) { - return ""; - } - } - - /** - * Waits for HBase installation's base (parent) znode to become available. - * @throws IOException on ZK errors - */ - public static void waitForBaseZNode(Configuration conf) throws IOException { - LOG.info("Waiting until the base znode is available"); - String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf), - conf.getInt(HConstants.ZK_SESSION_TIMEOUT, - HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance); - - final int maxTimeMs = 10000; - final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS; - - KeeperException keeperEx = null; - try { - try { - for (int attempt = 0; attempt < maxNumAttempts; ++attempt) { - try { - if (zk.exists(parentZNode, false) != null) { - LOG.info("Parent znode exists: " + parentZNode); - keeperEx = null; - break; - } - } catch (KeeperException e) { - keeperEx = e; - } - Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS); - } - } finally { - zk.close(); - } - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - } - - if (keeperEx != null) { - throw new IOException(keeperEx); - } - } - - /** - * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}. - * Used when can't let a {@link DeserializationException} out w/o changing public API. - * @param e Exception to convert - * @return Converted exception - */ - public static KeeperException convert(final DeserializationException e) { - KeeperException ke = new KeeperException.DataInconsistencyException(); - ke.initCause(e); - return ke; - } - - /** - * Recursively print the current state of ZK (non-transactional) - * @param root name of the root directory in zk to print - */ - public static void logZKTree(ZooKeeperWatcher zkw, String root) { - if (!LOG.isDebugEnabled()) return; - LOG.debug("Current zk system:"); - String prefix = "|-"; - LOG.debug(prefix + root); - try { - logZKTree(zkw, root, prefix); - } catch (KeeperException e) { - throw new RuntimeException(e); - } - } - - /** - * Helper method to print the current state of the ZK tree. - * @see #logZKTree(ZooKeeperWatcher, String) - * @throws KeeperException if an unexpected exception occurs - */ - protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) - throws KeeperException { - List children = ZKUtil.listChildrenNoWatch(zkw, root); - if (children == null) return; - for (String child : children) { - LOG.debug(prefix + child); - String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child); - logZKTree(zkw, node, prefix + "---"); - } - } - /** * @param position * @return Serialized protobuf of position with pb magic prefix prepended suitable @@ -2140,4 +327,19 @@ public class ZKUtil { } return storeIds; } + + /** + * @param bytes - Content of a failed region server or recovering region znode. + * @return long - The last flushed sequence Id for the region server + */ + public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) { + long lastRecordedFlushedSequenceId = -1l; + try { + lastRecordedFlushedSequenceId = parseWALPositionFrom(bytes); + } catch (DeserializationException e) { + lastRecordedFlushedSequenceId = -1l; + LOG.warn("Can't parse last flushed sequence Id", e); + } + return lastRecordedFlushedSequenceId; + } } diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index 135e720557..fbe869b3df 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -238,8 +238,8 @@ compile
- com.google.protobuf - protobuf-java + org.apache.hbase.thirdparty + hbase-shaded-protobuf diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Abortable.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java rename to hbase-common/src/main/java/org/apache/hadoop/hbase/Abortable.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 45f22a67ca..6bc4a83a03 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1029,6 +1029,8 @@ public final class HConstants { public static final String META_REPLICAS_NUM = "hbase.meta.replica.count"; public static final int DEFAULT_META_REPLICA_NUM = 1; + public static final int DEFAULT_REPLICA_ID = 0; + /** * The name of the configuration parameter that specifies * the number of bytes in a newly created checksum chunk. diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufHelpers.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufHelpers.java new file mode 100644 index 0000000000..4ff4551902 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufHelpers.java @@ -0,0 +1,25 @@ +package org.apache.hadoop.hbase.protobuf; + +import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message; + +import java.io.IOException; + +public class ProtobufHelpers { + /** + * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding + * buffers when working with byte arrays + * @param builder current message builder + * @param b byte array + * @param offset + * @param length + * @throws IOException + */ + public static void mergeFrom(Message.Builder builder, byte[] b, int offset, int length) + throws IOException { + final CodedInputStream codedInput = CodedInputStream.newInstance(b, offset, length); + codedInput.setSizeLimit(length); + builder.mergeFrom(codedInput); + codedInput.checkLastTagWas(0); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java similarity index 66% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java rename to hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java index 5268dafb8a..72227ad0d7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java @@ -17,17 +17,17 @@ */ package org.apache.hadoop.hbase.protobuf; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; +import static org.apache.hadoop.hbase.util.Bytes.compareTo; + /** * Protobufs utility. */ @InterfaceAudience.Private public class ProtobufMagic { - - private ProtobufMagic() { - } - /** * Magic we put ahead of a serialized protobuf message. * For example, all znode content is protobuf messages with the below magic @@ -44,30 +44,28 @@ public class ProtobufMagic { return isPBMagicPrefix(bytes, 0, bytes.length); } - /* - * Copied from Bytes.java to here - * hbase-common now depends on hbase-protocol - * Referencing Bytes.java directly would create circular dependency + /** + * Prepend the passed bytes with four bytes of magic, {@link ProtobufMagic#PB_MAGIC}, + * to flag what follows as a protobuf in hbase. Prepend these bytes to all content written to + * znodes, etc. + * @param bytes Bytes to decorate + * @return The passed bytes with magic prepended (Creates a new + * byte array that is bytes.length plus {@link ProtobufMagic#PB_MAGIC}.length. */ - private static int compareTo(byte[] buffer1, int offset1, int length1, - byte[] buffer2, int offset2, int length2) { - // Short circuit equal case - if (buffer1 == buffer2 && - offset1 == offset2 && - length1 == length2) { - return 0; - } - // Bring WritableComparator code local - int end1 = offset1 + length1; - int end2 = offset2 + length2; - for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) { - int a = (buffer1[i] & 0xff); - int b = (buffer2[j] & 0xff); - if (a != b) { - return a - b; - } + public static byte [] prependPBMagic(final byte [] bytes) { + return Bytes.add(PB_MAGIC, bytes); + } + + + /** + * @param bytes bytes to check + * @throws DeserializationException if we are missing the pb magic prefix + */ + public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException { + if (!isPBMagicPrefix(bytes)) { + throw new DeserializationException("Missing pb magic " + + Bytes.toString(PB_MAGIC) + " prefix"); } - return length1 - length2; } /** @@ -87,4 +85,4 @@ public class ProtobufMagic { public static int lengthOfPBMagic() { return PB_MAGIC.length; } -} + } diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadEndpointClient.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadEndpointClient.java index d32e6ea098..61561759ac 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadEndpointClient.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadEndpointClient.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-examples/pom.xml b/hbase-examples/pom.xml index 9f32fec1de..89b3d6c3e0 100644 --- a/hbase-examples/pom.xml +++ b/hbase-examples/pom.xml @@ -130,6 +130,10 @@ org.apache.hbase hbase-common + + org.apache.hbase + hbase-zookeeper + org.apache.hbase hbase-protocol diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml index 42c8da7e35..1e1faf2568 100644 --- a/hbase-it/pom.xml +++ b/hbase-it/pom.xml @@ -190,6 +190,11 @@ hbase-common jar + + org.apache.hbase + hbase-zookeeper + jar + org.apache.hbase hbase-protocol diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java index 8c0d273982..797b89b202 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java @@ -26,6 +26,7 @@ import java.util.TreeSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterManager.ServiceType; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java index 667daa8e54..7157fed134 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.TestMetaWithReplicas; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.testclassification.IntegrationTests; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -36,7 +36,7 @@ import org.junit.experimental.categories.Category; /** * An integration test that starts the cluster with three replicas for the meta * It then creates a table, flushes the meta, kills the server holding the primary. - * After that a client issues put/get requests on the created table - the other + * After that a client issues put/get requests on the created table - the other * replicas of the meta would be used to get the location of the region of the created * table. */ @@ -63,10 +63,10 @@ public class IntegrationTestMetaReplicas { Configuration conf = util.getConfiguration(); String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String primaryMetaZnode = ZKUtil.joinZNode(baseZNode, + String primaryMetaZnode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server")); // check that the data in the znode is parseable (this would also mean the znode exists) - byte[] data = ZKUtil.getData(zkw, primaryMetaZnode); + byte[] data = ZooKeeperUtil.getData(zkw, primaryMetaZnode); ProtobufUtil.toServerName(data); waitUntilZnodeAvailable(1); waitUntilZnodeAvailable(2); @@ -83,7 +83,7 @@ public class IntegrationTestMetaReplicas { String znode = util.getZooKeeperWatcher().znodePaths.getZNodeForReplica(replicaId); int i = 0; while (i < 1000) { - if (ZKUtil.checkExists(util.getZooKeeperWatcher(), znode) == -1) { + if (ZooKeeperUtil.checkExists(util.getZooKeeperWatcher(), znode) == -1) { Thread.sleep(100); i++; } else break; 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 0d85e4273b..9941c0d678 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 @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.util.ToolRunner; import org.apache.zookeeper.KeeperException; @@ -139,7 +139,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool { private void testZNodeACLs() throws IOException, KeeperException, InterruptedException { ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "IntegrationTestZnodeACLs", null); - RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher); + RecoverableZooKeeper zk = ZooKeeperUtil.connect(this.conf, watcher); String baseZNode = watcher.znodePaths.baseZNode; @@ -165,7 +165,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool { List children = zk.getChildren(znode, false); for (String child : children) { - checkZnodePermsRecursive(watcher, zk, ZKUtil.joinZNode(znode, child)); + checkZnodePermsRecursive(watcher, zk, ZooKeeperUtil.joinZNode(znode, child)); } } catch (KeeperException ke) { // if we are not authenticated for listChildren, it is fine. diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index 3bb966ac53..c7fbc3859b 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml index a56a470c81..a12bdf6cce 100644 --- a/hbase-replication/pom.xml +++ b/hbase-replication/pom.xml @@ -125,6 +125,10 @@ org.apache.hbase hbase-common + + org.apache.hbase + hbase-zookeeper + org.apache.hbase diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java index 2de61cba2b..ec595598bc 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java @@ -30,12 +30,12 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -86,7 +86,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase try { this.readPeerStateZnode(); } catch (DeserializationException e) { - throw ZKUtil.convert(e); + throw ZooKeeperUtil.convert(e); } } @@ -244,11 +244,11 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase */ private boolean ensurePeerEnabled(final String path) throws NodeExistsException, KeeperException { - if (ZKUtil.checkExists(zookeeper, path) == -1) { + if (ZooKeeperUtil.checkExists(zookeeper, path) == -1) { // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the // peer-state znode. This happens while adding a peer. // The peer state data is set as "ENABLED" by default. - ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path, + ZooKeeperUtil.createNodeIfNotExistsAndWatch(zookeeper, path, ReplicationStateZKBase.ENABLED_ZNODE_BYTES); return true; } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java index ca545f7880..90ea2930bd 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -42,8 +43,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -94,8 +94,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re @Override public void init() throws ReplicationException { try { - if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.peersZNode); + if (ZooKeeperUtil.checkExists(this.zookeeper, this.peersZNode) < 0) { + ZooKeeperUtil.createWithParents(this.zookeeper, this.peersZNode); } } catch (KeeperException e) { throw new ReplicationException("Could not initialize replication peers", e); @@ -126,18 +126,18 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re checkQueuesDeleted(id); - ZKUtil.createWithParents(this.zookeeper, this.peersZNode); + ZooKeeperUtil.createWithParents(this.zookeeper, this.peersZNode); List listOfOps = new ArrayList<>(2); - ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id), + ZKUtilOp op1 = ZooKeeperUtil.ZKUtilOp.createAndFailSilent(getPeerNode(id), ReplicationSerDeHelper.toByteArray(peerConfig)); // b/w PeerWatcher and ReplicationZookeeper#add method to create the // peer-state znode. This happens while adding a peer // The peer state data is set as "ENABLED" by default. - ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES); + ZKUtilOp op2 = ZooKeeperUtil.ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES); listOfOps.add(op1); listOfOps.add(op2); - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); + ZooKeeperUtil.multiOrSequential(this.zookeeper, listOfOps, false); // A peer is enabled by default } catch (KeeperException e) { throw new ReplicationException("Could not add peer with id=" + id @@ -152,7 +152,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re throw new IllegalArgumentException("Cannot remove peer with id=" + id + " because that id does not exist."); } - ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)); + ZooKeeperUtil + .deleteNodeRecursively(this.zookeeper, ZooKeeperUtil.joinZNode(this.peersZNode, id)); } catch (KeeperException e) { throw new ReplicationException("Could not remove peer with id=" + id, e); } @@ -204,7 +205,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id); } rpc.setTableCFsMap(tableCFs); - ZKUtil.setData(this.zookeeper, getPeerNode(id), + ZooKeeperUtil.setData(this.zookeeper, getPeerNode(id), ReplicationSerDeHelper.toByteArray(rpc)); LOG.info("Peer tableCFs with id= " + id + " is now " + ReplicationSerDeHelper.convertToString(tableCFs)); @@ -230,7 +231,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } String peerStateZNode = getPeerStateNode(id); try { - return ReplicationPeerZKImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode)); + return ReplicationPeerZKImpl.isStateEnabled( + ZooKeeperUtil.getData(this.zookeeper, peerStateZNode)); } catch (KeeperException e) { throw new ReplicationException(e); } catch (DeserializationException e) { @@ -249,7 +251,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re Map peers = new TreeMap<>(); List ids = null; try { - ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); + ids = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); for (String id : ids) { ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id); if (peerConfig == null) { @@ -286,7 +288,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re String znode = getPeerNode(peerId); byte[] data = null; try { - data = ZKUtil.getData(this.zookeeper, znode); + data = ZooKeeperUtil.getData(this.zookeeper, znode); } catch (InterruptedException e) { LOG.warn("Could not get configuration for peer because the thread " + "was interrupted. peerId=" + peerId); @@ -370,7 +372,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re existingConfig.setBandwidth(newConfig.getBandwidth()); try { - ZKUtil.setData(this.zookeeper, getPeerNode(id), + ZooKeeperUtil.setData(this.zookeeper, getPeerNode(id), ReplicationSerDeHelper.toByteArray(existingConfig)); } catch(KeeperException ke){ @@ -386,7 +388,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re public List getAllPeerIds() { List ids = null; try { - ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode); + ids = ZooKeeperUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode); } catch (KeeperException e) { this.abortable.abort("Cannot get the list of peers ", e); } @@ -400,7 +402,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re private void addExistingPeers() throws ReplicationException { List znodes = null; try { - znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); + znodes = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); } catch (KeeperException e) { throw new ReplicationException("Error getting the list of peer clusters.", e); } @@ -473,10 +475,10 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re byte[] stateBytes = (state == ReplicationProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES; - if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) { - ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes); + if (ZooKeeperUtil.checkExists(this.zookeeper, peerStateZNode) != -1) { + ZooKeeperUtil.setData(this.zookeeper, peerStateZNode, stateBytes); } else { - ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes); + ZooKeeperUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes); } LOG.info("Peer with id= " + id + " is now " + state.name()); } catch (KeeperException e) { @@ -534,7 +536,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } } // Check for hfile-refs queue - if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode) + if (-1 != ZooKeeperUtil.checkExists(zookeeper, hfileRefsZNode) && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) { throw new ReplicationException("Undeleted queue for peerId: " + peerId + ", found in hfile-refs node path " + hfileRefsZNode); diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java index 95b2e04d2e..1ffd24076a 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java @@ -25,10 +25,10 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; @@ -51,8 +51,8 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem @Override public void init() throws ReplicationException { try { - if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.queuesZNode); + if (ZooKeeperUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) { + ZooKeeperUtil.createWithParents(this.zookeeper, this.queuesZNode); } } catch (KeeperException e) { throw new ReplicationException("Internal error while initializing a queues client", e); @@ -61,11 +61,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem @Override public List getLogsInQueue(String serverName, String queueId) throws KeeperException { - String znode = ZKUtil.joinZNode(this.queuesZNode, serverName); - znode = ZKUtil.joinZNode(znode, queueId); + String znode = ZooKeeperUtil.joinZNode(this.queuesZNode, serverName); + znode = ZooKeeperUtil.joinZNode(znode, queueId); List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of wals for queueId=" + queueId + " and serverName=" + serverName, e); @@ -76,10 +76,10 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem @Override public List getAllQueues(String serverName) throws KeeperException { - String znode = ZKUtil.joinZNode(this.queuesZNode, serverName); + String znode = ZooKeeperUtil.joinZNode(this.queuesZNode, serverName); List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e); throw e; @@ -128,7 +128,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem public int getQueuesZNodeCversion() throws KeeperException { try { Stat stat = new Stat(); - ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat); + ZooKeeperUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat); return stat.getCversion(); } catch (KeeperException e) { this.abortable.abort("Failed to get stat of replication rs node", e); @@ -140,7 +140,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem public int getHFileRefsNodeChangeVersion() throws KeeperException { Stat stat = new Stat(); try { - ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat); + ZooKeeperUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat); } catch (KeeperException e) { this.abortable.abort("Failed to get stat of replication hfile references node.", e); throw e; @@ -152,7 +152,7 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem public List getAllPeersFromHFileRefsQueue() throws KeeperException { List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of all peers in hfile references node.", e); throw e; @@ -162,10 +162,10 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem @Override public List getReplicableHFiles(String peerId) throws KeeperException { - String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId); + String znode = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, peerId); List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e); throw e; diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java index 8e61df9419..a0f3a075ef 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java @@ -25,18 +25,17 @@ import java.util.TreeSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -81,10 +80,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void init(String serverName) throws ReplicationException { - this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName); + this.myQueuesZnode = ZooKeeperUtil.joinZNode(this.queuesZNode, serverName); try { - if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode); + if (ZooKeeperUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) { + ZooKeeperUtil.createWithParents(this.zookeeper, this.myQueuesZnode); } } catch (KeeperException e) { throw new ReplicationException("Could not initialize replication queues.", e); @@ -92,8 +91,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) { try { - if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode); + if (ZooKeeperUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) { + ZooKeeperUtil.createWithParents(this.zookeeper, this.hfileRefsZNode); } } catch (KeeperException e) { throw new ReplicationException("Could not initialize hfile references replication queue.", @@ -105,7 +104,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removeQueue(String queueId) { try { - ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId)); + ZooKeeperUtil.deleteNodeRecursively(this.zookeeper, ZooKeeperUtil + .joinZNode(this.myQueuesZnode, queueId)); } catch (KeeperException e) { this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e); } @@ -113,10 +113,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void addLog(String queueId, String filename) throws ReplicationException { - String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId); - znode = ZKUtil.joinZNode(znode, filename); + String znode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, queueId); + znode = ZooKeeperUtil.joinZNode(znode, filename); try { - ZKUtil.createWithParents(this.zookeeper, znode); + ZooKeeperUtil.createWithParents(this.zookeeper, znode); } catch (KeeperException e) { throw new ReplicationException( "Could not add log because znode could not be created. queueId=" + queueId @@ -127,9 +127,9 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removeLog(String queueId, String filename) { try { - String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId); - znode = ZKUtil.joinZNode(znode, filename); - ZKUtil.deleteNode(this.zookeeper, znode); + String znode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, queueId); + znode = ZooKeeperUtil.joinZNode(znode, filename); + ZooKeeperUtil.deleteNode(this.zookeeper, znode); } catch (KeeperException e) { this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename=" + filename + ")", e); @@ -139,10 +139,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void setLogPosition(String queueId, String filename, long position) { try { - String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId); - znode = ZKUtil.joinZNode(znode, filename); + String znode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, queueId); + znode = ZooKeeperUtil.joinZNode(znode, filename); // Why serialize String of Long and not Long as bytes? - ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position)); + ZooKeeperUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position)); } catch (KeeperException e) { this.abortable.abort("Failed to write replication wal position (filename=" + filename + ", position=" + position + ")", e); @@ -151,11 +151,11 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public long getLogPosition(String queueId, String filename) throws ReplicationException { - String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId); - String znode = ZKUtil.joinZNode(clusterZnode, filename); + String clusterZnode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, queueId); + String znode = ZooKeeperUtil.joinZNode(clusterZnode, filename); byte[] bytes = null; try { - bytes = ZKUtil.getData(this.zookeeper, znode); + bytes = ZooKeeperUtil.getData(this.zookeeper, znode); } catch (KeeperException e) { throw new ReplicationException("Internal Error: could not get position in log for queueId=" + queueId + ", filename=" + filename, e); @@ -176,7 +176,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public boolean isThisOurRegionServer(String regionserver) { - return ZKUtil.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode); + return ZooKeeperUtil.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode); } @Override @@ -184,10 +184,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R if (isThisOurRegionServer(regionserver)) { return null; } - String rsZnodePath = ZKUtil.joinZNode(this.queuesZNode, regionserver); + String rsZnodePath = ZooKeeperUtil.joinZNode(this.queuesZNode, regionserver); List queues = null; try { - queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath); + queues = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath); } catch (KeeperException e) { this.abortable.abort("Failed to getUnClaimedQueueIds for RS" + regionserver, e); } @@ -202,11 +202,11 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removeReplicatorIfQueueIsEmpty(String regionserver) { - String rsPath = ZKUtil.joinZNode(this.queuesZNode, regionserver); + String rsPath = ZooKeeperUtil.joinZNode(this.queuesZNode, regionserver); try { - List list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath); + List list = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, rsPath); if (list != null && list.isEmpty()){ - ZKUtil.deleteNode(this.zookeeper, rsPath); + ZooKeeperUtil.deleteNode(this.zookeeper, rsPath); } } catch (KeeperException e) { LOG.warn("Got error while removing replicator", e); @@ -216,7 +216,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removeAllQueues() { try { - ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode); + ZooKeeperUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode); } catch (KeeperException e) { // if the znode is already expired, don't bother going further if (e instanceof KeeperException.SessionExpiredException) { @@ -229,10 +229,10 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public List getLogsInQueue(String queueId) { - String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId); + String znode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, queueId); List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, znode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e); } @@ -243,7 +243,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R public List getAllQueues() { List listOfQueues = null; try { - listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode); + listOfQueues = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode); } catch (KeeperException e) { this.abortable.abort("Failed to get a list of queues for region server: " + this.myQueuesZnode, e); @@ -260,52 +260,52 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R private Pair> moveQueueUsingMulti(String znode, String peerId) { try { // hbase/replication/rs/deadrs - String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode); + String deadRSZnodePath = ZooKeeperUtil.joinZNode(this.queuesZNode, znode); List listOfOps = new ArrayList<>(); ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId); String newPeerId = peerId + "-" + znode; - String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId); + String newPeerZnode = ZooKeeperUtil.joinZNode(this.myQueuesZnode, newPeerId); // check the logs queue for the old peer cluster - String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId); - List wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode); + String oldClusterZnode = ZooKeeperUtil.joinZNode(deadRSZnodePath, peerId); + List wals = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode); if (!peerExists(replicationQueueInfo.getPeerId())) { LOG.warn("Peer " + replicationQueueInfo.getPeerId() + " didn't exist, will move its queue to avoid the failure of multi op"); for (String wal : wals) { - String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal); - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); + String oldWalZnode = ZooKeeperUtil.joinZNode(oldClusterZnode, wal); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); } - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); + ZooKeeperUtil.multiOrSequential(this.zookeeper, listOfOps, false); return null; } SortedSet logQueue = new TreeSet<>(); if (wals == null || wals.isEmpty()) { - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); } else { // create the new cluster znode - ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY); + ZKUtilOp op = ZooKeeperUtil.ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY); listOfOps.add(op); // get the offset of the logs and set it to new znodes for (String wal : wals) { - String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal); - byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode); + String oldWalZnode = ZooKeeperUtil.joinZNode(oldClusterZnode, wal); + byte[] logOffset = ZooKeeperUtil.getData(this.zookeeper, oldWalZnode); LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset)); - String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal); - listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset)); - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); + String newLogZnode = ZooKeeperUtil.joinZNode(newPeerZnode, wal); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(newLogZnode, logOffset)); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); logQueue.add(wal); } // add delete op for peer - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); if (LOG.isTraceEnabled()) LOG.trace(" The multi list size is: " + listOfOps.size()); } - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); + ZooKeeperUtil.multiOrSequential(this.zookeeper, listOfOps, false); LOG.info("Atomically moved " + znode + "/" + peerId + "'s WALs to my queue"); return new Pair<>(newPeerId, logQueue); @@ -322,7 +322,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void addHFileRefs(String peerId, List> pairs) throws ReplicationException { - String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId); + String peerZnode = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, peerId); boolean debugEnabled = LOG.isDebugEnabled(); if (debugEnabled) { LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode); @@ -332,8 +332,8 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R List listOfOps = new ArrayList<>(size); for (int i = 0; i < size; i++) { - listOfOps.add(ZKUtilOp.createAndFailSilent( - ZKUtil.joinZNode(peerZnode, pairs.get(i).getSecond().getName()), + listOfOps.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent( + ZooKeeperUtil.joinZNode(peerZnode, pairs.get(i).getSecond().getName()), HConstants.EMPTY_BYTE_ARRAY)); } if (debugEnabled) { @@ -341,7 +341,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R + " is " + listOfOps.size()); } try { - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); + ZooKeeperUtil.multiOrSequential(this.zookeeper, listOfOps, true); } catch (KeeperException e) { throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e); } @@ -349,7 +349,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removeHFileRefs(String peerId, List files) { - String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId); + String peerZnode = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, peerId); boolean debugEnabled = LOG.isDebugEnabled(); if (debugEnabled) { LOG.debug("Removing hfile references " + files + " from queue " + peerZnode); @@ -359,14 +359,15 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R List listOfOps = new ArrayList<>(size); for (int i = 0; i < size; i++) { - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)))); + listOfOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent( + ZooKeeperUtil.joinZNode(peerZnode, files.get(i)))); } if (debugEnabled) { LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode + " is " + listOfOps.size()); } try { - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); + ZooKeeperUtil.multiOrSequential(this.zookeeper, listOfOps, true); } catch (KeeperException e) { LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e); } @@ -374,11 +375,11 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void addPeerToHFileRefs(String peerId) throws ReplicationException { - String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId); + String peerZnode = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, peerId); try { - if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) { + if (ZooKeeperUtil.checkExists(this.zookeeper, peerZnode) == -1) { LOG.info("Adding peer " + peerId + " to hfile reference queue."); - ZKUtil.createWithParents(this.zookeeper, peerZnode); + ZooKeeperUtil.createWithParents(this.zookeeper, peerZnode); } } catch (KeeperException e) { throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.", @@ -388,16 +389,16 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R @Override public void removePeerFromHFileRefs(String peerId) { - final String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId); + final String peerZnode = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, peerId); try { - if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) { + if (ZooKeeperUtil.checkExists(this.zookeeper, peerZnode) == -1) { if (LOG.isDebugEnabled()) { LOG.debug("Peer " + peerZnode + " not found in hfile reference queue."); } return; } else { LOG.info("Removing peer " + peerZnode + " from hfile reference queue."); - ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode); + ZooKeeperUtil.deleteNodeRecursively(this.zookeeper, peerZnode); } } catch (KeeperException e) { LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.", diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java index c6501e189d..30142c1fa2 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java @@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -87,17 +87,17 @@ 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.znodePaths.baseZNode, + this.replicationZNode = ZooKeeperUtil.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName); - this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName); - this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName); - this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName); + this.peersZNode = ZooKeeperUtil.joinZNode(replicationZNode, peersZNodeName); + this.queuesZNode = ZooKeeperUtil.joinZNode(replicationZNode, queuesZNodeName); + this.hfileRefsZNode = ZooKeeperUtil.joinZNode(replicationZNode, hfileRefsZNodeName); } public List getListOfReplicators() { List result = null; try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode); + result = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode); } catch (KeeperException e) { this.abortable.abort("Failed to get list of replicators", e); } @@ -127,7 +127,7 @@ public abstract class ReplicationStateZKBase { } protected boolean peerExists(String id) throws KeeperException { - return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0; + return ZooKeeperUtil.checkExists(this.zookeeper, ZooKeeperUtil.joinZNode(this.peersZNode, id)) >= 0; } /** @@ -141,15 +141,15 @@ public abstract class ReplicationStateZKBase { @VisibleForTesting protected String getTableCFsNode(String id) { - return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName)); + return ZooKeeperUtil.joinZNode(this.peersZNode, ZooKeeperUtil.joinZNode(id, this.tableCFsNodeName)); } @VisibleForTesting protected String getPeerStateNode(String id) { - return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName)); + return ZooKeeperUtil.joinZNode(this.peersZNode, ZooKeeperUtil.joinZNode(id, this.peerStateNodeName)); } @VisibleForTesting protected String getPeerNode(String id) { - return ZKUtil.joinZNode(this.peersZNode, id); + return ZooKeeperUtil.joinZNode(this.peersZNode, id); } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java index ade1c4d42e..02d1b535ca 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java @@ -24,11 +24,11 @@ import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -241,7 +241,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements private List getRegisteredRegionServers() { List result = null; try { - result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode); + result = ZooKeeperUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode); } catch (KeeperException e) { this.abortable.abort("Get list of registered region servers", e); } 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 9520f5fbf8..870a5a7579 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 @@ -74,7 +74,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.security.access.AccessControlLists; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -313,13 +313,13 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager { } List retrieveGroupListFromZookeeper() throws IOException { - String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); + String groupBasePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); List RSGroupInfoList = Lists.newArrayList(); //Overwrite any info stored by table, this takes precedence try { - if(ZKUtil.checkExists(watcher, groupBasePath) != -1) { - for(String znode: ZKUtil.listChildrenAndWatchForNewChildren(watcher, groupBasePath)) { - byte[] data = ZKUtil.getData(watcher, ZKUtil.joinZNode(groupBasePath, znode)); + if(ZooKeeperUtil.checkExists(watcher, groupBasePath) != -1) { + for(String znode: ZooKeeperUtil.listChildrenAndWatchForNewChildren(watcher, groupBasePath)) { + byte[] data = ZooKeeperUtil.getData(watcher, ZooKeeperUtil.joinZNode(groupBasePath, znode)); if(data.length > 0) { ProtobufUtil.expectPBMagicPrefix(data); ByteArrayInputStream bis = new ByteArrayInputStream( @@ -463,30 +463,30 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager { resetRSGroupAndTableMaps(newGroupMap, newTableMap); try { - String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); - ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC); + String groupBasePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); + ZooKeeperUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC); - List zkOps = new ArrayList<>(newGroupMap.size()); + List zkOps = new ArrayList<>(newGroupMap.size()); for(String groupName : prevRSGroups) { if(!newGroupMap.containsKey(groupName)) { - String znode = ZKUtil.joinZNode(groupBasePath, groupName); - zkOps.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(znode)); + String znode = ZooKeeperUtil.joinZNode(groupBasePath, groupName); + zkOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(znode)); } } for (RSGroupInfo RSGroupInfo : newGroupMap.values()) { - String znode = ZKUtil.joinZNode(groupBasePath, RSGroupInfo.getName()); + String znode = ZooKeeperUtil.joinZNode(groupBasePath, RSGroupInfo.getName()); RSGroupProtos.RSGroupInfo proto = RSGroupProtobufUtil.toProtoGroupInfo(RSGroupInfo); LOG.debug("Updating znode: "+znode); - ZKUtil.createAndFailSilent(watcher, znode); - zkOps.add(ZKUtil.ZKUtilOp.deleteNodeFailSilent(znode)); - zkOps.add(ZKUtil.ZKUtilOp.createAndFailSilent(znode, + ZooKeeperUtil.createAndFailSilent(watcher, znode); + zkOps.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(znode)); + zkOps.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(znode, ProtobufUtil.prependPBMagic(proto.toByteArray()))); } LOG.debug("Writing ZK GroupInfo count: " + zkOps.size()); - ZKUtil.multiOrSequential(watcher, zkOps, false); + ZooKeeperUtil.multiOrSequential(watcher, zkOps, false); } catch (KeeperException e) { LOG.error("Failed to write to rsGroupZNode", e); masterServices.abort("Failed to write to rsGroupZNode", e); @@ -524,7 +524,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager { LOG.debug("Reading online RS from zookeeper"); List servers = new LinkedList<>(); try { - for (String el: ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode)) { + for (String el: ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode)) { servers.add(ServerName.parseServerName(el)); } } catch (KeeperException e) { @@ -788,12 +788,12 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager { = MultiRowMutationProtos.MutateRowsRequest.newBuilder(); for (Mutation mutation : mutations) { if (mutation instanceof Put) { - mmrBuilder.addMutationRequest(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation( + mmrBuilder.addMutationRequest(ProtobufUtil.toMutation( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.PUT, mutation)); } else if (mutation instanceof Delete) { mmrBuilder.addMutationRequest( - org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation( + ProtobufUtil.toMutation( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto. MutationType.DELETE, mutation)); } else { 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 ea019bc745..39d63a6e1c 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Result; @@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.Assert; @@ -124,9 +124,9 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin { Assert.assertEquals(Sets.newHashSet(groupMap.values()), Sets.newHashSet(wrapped.listRSGroups())); try { - String groupBasePath = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "rsgroup"); - for(String znode: ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) { - byte[] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(groupBasePath, znode)); + String groupBasePath = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "rsgroup"); + for(String znode: ZooKeeperUtil.listChildrenNoWatch(zkw, groupBasePath)) { + byte[] data = ZooKeeperUtil.getData(zkw, ZooKeeperUtil.joinZNode(groupBasePath, znode)); if(data.length > 0) { ProtobufUtil.expectPBMagicPrefix(data); ByteArrayInputStream bis = new ByteArrayInputStream( diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index ea02f26efc..c17135f7c9 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -423,6 +423,16 @@ org.apache.hbase hbase-metrics + + org.apache.hbase + hbase-zookeeper + + + org.apache.hbase + hbase-zookeeper + test-jar + test + org.apache.hbase 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 6e40295940..0729dbce1f 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 @@ -21,11 +21,11 @@ package org.apache.hadoop.hbase; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -61,14 +61,14 @@ public class ZKNamespaceManager extends ZooKeeperListener { public void start() throws IOException { watcher.registerListener(this); try { - if (ZKUtil.watchAndCheckExists(watcher, nsZNode)) { - List existing = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); + if (ZooKeeperUtil.watchAndCheckExists(watcher, nsZNode)) { + List existing = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); if (existing != null) { refreshNodes(existing); } } else { - ZKUtil.createWithParents(watcher, nsZNode); + ZooKeeperUtil.createWithParents(watcher, nsZNode); } } catch (KeeperException e) { throw new IOException("Failed to initialize ZKNamespaceManager", e); @@ -102,8 +102,8 @@ public class ZKNamespaceManager extends ZooKeeperListener { public void nodeCreated(String path) { if (nsZNode.equals(path)) { try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); refreshNodes(nodes); } catch (KeeperException ke) { String msg = "Error reading data from zookeeper"; @@ -119,17 +119,17 @@ public class ZKNamespaceManager extends ZooKeeperListener { @Override public void nodeDeleted(String path) { - if (nsZNode.equals(ZKUtil.getParent(path))) { - String nsName = ZKUtil.getNodeName(path); + if (nsZNode.equals(ZooKeeperUtil.getParent(path))) { + String nsName = ZooKeeperUtil.getNodeName(path); cache.remove(nsName); } } @Override public void nodeDataChanged(String path) { - if (nsZNode.equals(ZKUtil.getParent(path))) { + if (nsZNode.equals(ZooKeeperUtil.getParent(path))) { try { - byte[] data = ZKUtil.getDataAndWatch(watcher, path); + byte[] data = ZooKeeperUtil.getDataAndWatch(watcher, path); NamespaceDescriptor ns = ProtobufUtil.toNamespaceDescriptor( HBaseProtos.NamespaceDescriptor.parseFrom(data)); @@ -151,8 +151,8 @@ public class ZKNamespaceManager extends ZooKeeperListener { public void nodeChildrenChanged(String path) { if (nsZNode.equals(path)) { try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode); refreshNodes(nodes); } catch (KeeperException ke) { LOG.error("Error reading data from zookeeper for path "+path, ke); @@ -165,9 +165,9 @@ public class ZKNamespaceManager extends ZooKeeperListener { } private void deleteNamespace(String name) throws IOException { - String zNode = ZKUtil.joinZNode(nsZNode, name); + String zNode = ZooKeeperUtil.joinZNode(nsZNode, name); try { - ZKUtil.deleteNode(watcher, zNode); + ZooKeeperUtil.deleteNode(watcher, zNode); } catch (KeeperException e) { if (e instanceof KeeperException.NoNodeException) { // If the node does not exist, it could be already deleted. Continue without fail. @@ -180,10 +180,10 @@ public class ZKNamespaceManager extends ZooKeeperListener { } private void writeNamespace(NamespaceDescriptor ns) throws IOException { - String zNode = ZKUtil.joinZNode(nsZNode, ns.getName()); + String zNode = ZooKeeperUtil.joinZNode(nsZNode, ns.getName()); try { - ZKUtil.createWithParents(watcher, zNode); - ZKUtil.updateExistingNodeData(watcher, zNode, + ZooKeeperUtil.createWithParents(watcher, zNode); + ZooKeeperUtil.updateExistingNodeData(watcher, zNode, ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray(), -1); } catch (KeeperException e) { LOG.error("Failed updating permissions for namespace "+ns.getName(), e); @@ -191,11 +191,11 @@ public class ZKNamespaceManager extends ZooKeeperListener { } } - private void refreshNodes(List nodes) throws IOException { - for (ZKUtil.NodeAndData n : nodes) { + private void refreshNodes(List nodes) throws IOException { + for (ZooKeeperUtil.NodeAndData n : nodes) { if (n.isEmpty()) continue; String path = n.getNode(); - String namespace = ZKUtil.getNodeName(path); + String namespace = ZooKeeperUtil.getNodeName(path); byte[] nodeData = n.getData(); if (LOG.isDebugEnabled()) { LOG.debug("Updating namespace cache from node "+namespace+" with data: "+ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java index bd8a58e467..2103a90da7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZNodeClearer.java @@ -31,7 +31,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -131,7 +131,7 @@ public class ZNodeClearer { * @param rsZnodePath from HBASE_ZNODE_FILE * @return String representation of ServerName or null if fails */ - + public static String parseMasterServerName(String rsZnodePath) { String masterServerName = null; try { @@ -140,12 +140,12 @@ public class ZNodeClearer { } catch (IndexOutOfBoundsException e) { LOG.warn("String " + rsZnodePath + " has wrong format", e); } - return masterServerName; + return masterServerName; } - + /** - * - * @return true if cluster is configured with master-rs collocation + * + * @return true if cluster is configured with master-rs collocation */ private static boolean tablesOnMaster(Configuration conf) { boolean tablesOnMaster = true; @@ -182,9 +182,9 @@ public class ZNodeClearer { try { znodeFileContent = ZNodeClearer.readMyEphemeralNodeOnDisk(); if(ZNodeClearer.tablesOnMaster(conf)) { - //In case of master crash also remove rsZnode since master is also regionserver - ZKUtil.deleteNodeFailSilent(zkw, znodeFileContent); - return MasterAddressTracker.deleteIfEquals(zkw, + //In case of master crash also remove rsZnode since master is also regionserver + ZooKeeperUtil.deleteNodeFailSilent(zkw, znodeFileContent); + return MasterAddressTracker.deleteIfEquals(zkw, ZNodeClearer.parseMasterServerName(znodeFileContent)); } else { return MasterAddressTracker.deleteIfEquals(zkw, znodeFileContent); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java index 61d6a0288e..d73a782676 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/HFileArchiveManager.java @@ -22,11 +22,11 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -86,7 +86,7 @@ class HFileArchiveManager { public HFileArchiveManager disableHFileBackup() throws IOException { LOG.debug("Disabling backups on all tables."); try { - ZKUtil.deleteNodeRecursively(this.zooKeeper, archiveZnode); + ZooKeeperUtil.deleteNodeRecursively(this.zooKeeper, archiveZnode); return this; } catch (KeeperException e) { throw new IOException("Unexpected ZK exception!", e); @@ -106,12 +106,12 @@ class HFileArchiveManager { private void enable(ZooKeeperWatcher zooKeeper, byte[] table) throws KeeperException { LOG.debug("Ensuring archiving znode exists"); - ZKUtil.createAndFailSilent(zooKeeper, archiveZnode); + ZooKeeperUtil.createAndFailSilent(zooKeeper, archiveZnode); // then add the table to the list of znodes to archive String tableNode = this.getTableNode(table); LOG.debug("Creating: " + tableNode + ", data: []"); - ZKUtil.createSetData(zooKeeper, tableNode, new byte[0]); + ZooKeeperUtil.createSetData(zooKeeper, tableNode, new byte[0]); } /** @@ -127,7 +127,7 @@ class HFileArchiveManager { zooKeeper.sync(archiveZnode); // if the top-level archive node is gone, then we are done - if (ZKUtil.checkExists(zooKeeper, archiveZnode) < 0) { + if (ZooKeeperUtil.checkExists(zooKeeper, archiveZnode) < 0) { return; } // delete the table node, from the archive @@ -136,7 +136,7 @@ class HFileArchiveManager { zooKeeper.sync(tableNode); LOG.debug("Attempting to delete table node:" + tableNode); - ZKUtil.deleteNodeRecursively(zooKeeper, tableNode); + ZooKeeperUtil.deleteNodeRecursively(zooKeeper, tableNode); } public void stop() { @@ -155,7 +155,7 @@ class HFileArchiveManager { */ public boolean isArchivingEnabled(byte[] table) throws KeeperException { String tableNode = this.getTableNode(table); - return ZKUtil.checkExists(zooKeeper, tableNode) >= 0; + return ZooKeeperUtil.checkExists(zooKeeper, tableNode) >= 0; } /** @@ -164,6 +164,6 @@ class HFileArchiveManager { * @return znode for the table's archive status */ private String getTableNode(byte[] table) { - return ZKUtil.joinZNode(archiveZnode, Bytes.toString(table)); + return ZooKeeperUtil.joinZNode(archiveZnode, Bytes.toString(table)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java index 72c292f244..7d665e2ceb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/TableHFileArchiveTracker.java @@ -22,10 +22,10 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -110,10 +110,10 @@ public class TableHFileArchiveTracker extends ZooKeeperListener { * @throws KeeperException if an unexpected zk exception occurs */ private void addAndReWatchTable(String tableZnode) throws KeeperException { - getMonitor().addTable(ZKUtil.getNodeName(tableZnode)); + getMonitor().addTable(ZooKeeperUtil.getNodeName(tableZnode)); // re-add a watch to the table created // and check to make sure it wasn't deleted - if (!ZKUtil.watchAndCheckExists(watcher, tableZnode)) { + if (!ZooKeeperUtil.watchAndCheckExists(watcher, tableZnode)) { safeStopTrackingTable(tableZnode); } } @@ -125,9 +125,9 @@ public class TableHFileArchiveTracker extends ZooKeeperListener { * @throws KeeperException if an unexpected zk exception occurs */ private void safeStopTrackingTable(String tableZnode) throws KeeperException { - getMonitor().removeTable(ZKUtil.getNodeName(tableZnode)); + getMonitor().removeTable(ZooKeeperUtil.getNodeName(tableZnode)); // if the table exists, then add and rewatch it - if (ZKUtil.checkExists(watcher, tableZnode) >= 0) { + if (ZooKeeperUtil.checkExists(watcher, tableZnode) >= 0) { addAndReWatchTable(tableZnode); } } @@ -155,7 +155,7 @@ public class TableHFileArchiveTracker extends ZooKeeperListener { // exists for that matter), so its better not to add unnecessary load to // zk for watches. If the table is created again, then we will get the // notification in childrenChanaged. - getMonitor().removeTable(ZKUtil.getNodeName(path)); + getMonitor().removeTable(ZooKeeperUtil.getNodeName(path)); } /** @@ -164,7 +164,7 @@ public class TableHFileArchiveTracker extends ZooKeeperListener { */ private void checkEnabledAndUpdate() { try { - if (ZKUtil.watchAndCheckExists(watcher, archiveHFileZNode)) { + if (ZooKeeperUtil.watchAndCheckExists(watcher, archiveHFileZNode)) { LOG.debug(archiveHFileZNode + " znode does exist, checking for tables to archive"); // update the tables we should backup, to get the most recent state. @@ -188,7 +188,7 @@ public class TableHFileArchiveTracker extends ZooKeeperListener { // get the children and watch for new children LOG.debug("Updating watches on tables to archive."); // get the children and add watches for each of the children - List tables = ZKUtil.listChildrenAndWatchThem(watcher, archiveHFileZNode); + List tables = ZooKeeperUtil.listChildrenAndWatchThem(watcher, archiveHFileZNode); LOG.debug("Starting archive for tables:" + tables); // if archiving is still enabled if (tables != null && tables.size() > 0) { 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 6c173cf0f9..56180ff35f 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 @@ -19,12 +19,12 @@ package org.apache.hadoop.hbase.backup.example; import java.io.IOException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -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.znodePaths.baseZNode, conf.get( + return ZooKeeperUtil.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 ef99de0847..77d345d5dc 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 @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.util.StringUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -140,7 +141,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements public int remainingTasksInCoordination() { int count = 0; try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); + List tasks = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (tasks != null) { int listSize = tasks.size(); for (int i = 0; i < listSize; i++) { @@ -292,7 +293,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements final String metaEncodeRegionName = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName(); int count = 0; try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); + List tasks = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (tasks != null) { int listSize = tasks.size(); for (int i = 0; i < listSize; i++) { @@ -310,7 +311,7 @@ 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, + List regions = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.recoveringRegionsZNode); if (regions != null) { int listSize = regions.size(); @@ -328,21 +329,22 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements continue; } } - String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, region); - List failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); + String nodePath = ZooKeeperUtil + .joinZNode(watcher.znodePaths.recoveringRegionsZNode, region); + List failedServers = ZooKeeperUtil.listChildrenNoWatch(watcher, nodePath); if (failedServers == null || failedServers.isEmpty()) { - ZKUtil.deleteNode(watcher, nodePath); + ZooKeeperUtil.deleteNode(watcher, nodePath); continue; } if (recoveredServerNameSet.containsAll(failedServers)) { - ZKUtil.deleteNodeRecursively(watcher, nodePath); + ZooKeeperUtil.deleteNodeRecursively(watcher, nodePath); } else { int tmpFailedServerSize = failedServers.size(); for (int j = 0; j < tmpFailedServerSize; j++) { String failedServer = failedServers.get(j); if (recoveredServerNameSet.contains(failedServer)) { - String tmpPath = ZKUtil.joinZNode(nodePath, failedServer); - ZKUtil.deleteNode(watcher, tmpPath); + String tmpPath = ZooKeeperUtil.joinZNode(nodePath, failedServer); + ZooKeeperUtil.deleteNode(watcher, tmpPath); } } } @@ -416,7 +418,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements private void createNode(String path, Long retry_count) { SplitLogTask slt = new SplitLogTask.Unassigned(details.getServerName(), getRecoveryMode()); - ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(), + ZooKeeperUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(), retry_count); SplitLogCounters.tot_mgr_node_create_queued.increment(); return; @@ -562,7 +564,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements private void lookForOrphans() { List orphans; try { - orphans = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.znodePaths.splitLogZNode); + orphans = ZooKeeperUtil.listChildrenNoWatch(this.watcher, this.watcher.znodePaths.splitLogZNode); if (orphans == null) { LOG.warn("could not get children of " + this.watcher.znodePaths.splitLogZNode); return; @@ -576,7 +578,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.znodePaths.splitLogZNode, path); + String nodepath = ZooKeeperUtil.joinZNode(watcher.znodePaths.splitLogZNode, path); if (ZKSplitLog.isRescanNode(watcher, nodepath)) { rescan_nodes++; LOG.debug("found orphan rescan node " + path); @@ -604,7 +606,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements long retries = this.zkretries; do { - String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, + String nodePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, regionEncodeName); long lastRecordedFlushedSequenceId = -1; try { @@ -616,25 +618,25 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements * znode layout: .../region_id[last known flushed sequence id]/failed server[last known * flushed sequence id for the server] */ - byte[] data = ZKUtil.getData(this.watcher, nodePath); + byte[] data = ZooKeeperUtil.getData(this.watcher, nodePath); if (data == null) { - ZKUtil + ZooKeeperUtil .createSetData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId)); } else { lastRecordedFlushedSequenceId = - ZKSplitLog.parseLastFlushedSequenceIdFrom(data); + ZKUtil.parseLastFlushedSequenceIdFrom(data); if (lastRecordedFlushedSequenceId < lastSequenceId) { // update last flushed sequence id in the region level - ZKUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId)); + ZooKeeperUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId)); } } // go one level deeper with server name - nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName()); + nodePath = ZooKeeperUtil.joinZNode(nodePath, serverName.getServerName()); if (lastSequenceId <= lastRecordedFlushedSequenceId) { // the newly assigned RS failed even before any flush to the region lastSequenceId = lastRecordedFlushedSequenceId; } - ZKUtil.createSetData(this.watcher, nodePath, + ZooKeeperUtil.createSetData(this.watcher, nodePath, ZKUtil.regionSequenceIdsToByteArray(lastSequenceId, null)); if (LOG.isDebugEnabled()) { LOG.debug("Marked " + regionEncodeName + " recovering from " + serverName + @@ -681,15 +683,15 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements throws IOException, InterruptedIOException { try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); + List tasks = ZooKeeperUtil.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.znodePaths.splitLogZNode, t)); + data = ZooKeeperUtil.getData(this.watcher, + ZooKeeperUtil.joinZNode(watcher.znodePaths.splitLogZNode, t)); } catch (InterruptedException e) { throw new InterruptedIOException(); } @@ -717,16 +719,16 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } // remove recovering regions which doesn't have any RS associated with it - List regions = ZKUtil.listChildrenNoWatch(watcher, + List regions = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.recoveringRegionsZNode); if (regions != null) { int listSize = regions.size(); for (int i = 0; i < listSize; i++) { - String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, + String nodePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, regions.get(i)); - List regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); + List regionFailedServers = ZooKeeperUtil.listChildrenNoWatch(watcher, nodePath); if (regionFailedServers == null || regionFailedServers.isEmpty()) { - ZKUtil.deleteNode(watcher, nodePath); + ZooKeeperUtil.deleteNode(watcher, nodePath); continue; } boolean needMoreRecovery = false; @@ -738,7 +740,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } } if (!needMoreRecovery) { - ZKUtil.deleteNodeRecursively(watcher, nodePath); + ZooKeeperUtil.deleteNodeRecursively(watcher, nodePath); } } } @@ -758,7 +760,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } private List listSplitLogTasks() throws KeeperException { - List taskOrRescanList = ZKUtil.listChildrenNoWatch(watcher, + List taskOrRescanList = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (taskOrRescanList == null || taskOrRescanList.isEmpty()) { return Collections. emptyList(); @@ -804,7 +806,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements // Firstly check if there are outstanding recovering regions try { - List regions = ZKUtil.listChildrenNoWatch(watcher, + List regions = ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.recoveringRegionsZNode); if (regions != null && !regions.isEmpty()) { hasRecoveringRegions = true; @@ -821,8 +823,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.znodePaths.splitLogZNode, task)); + byte[] data = ZooKeeperUtil.getData(this.watcher, + ZooKeeperUtil.joinZNode(watcher.znodePaths.splitLogZNode, task)); if (data == null) continue; SplitLogTask slt = SplitLogTask.parseFrom(data); previousRecoveryMode = slt.getMode(); @@ -882,7 +884,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements // blocking zk call but this is done from the timeout thread SplitLogTask slt = new SplitLogTask.Unassigned(this.details.getServerName(), getRecoveryMode()); - if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) { + if (ZooKeeperUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) { LOG.debug("failed to resubmit task " + path + " version changed"); return false; } 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 14e7796e16..85a2610646 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.coordination; import java.io.IOException; +import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,6 +38,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SplitLogCounters; import org.apache.hadoop.hbase.SplitLogTask; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; @@ -52,7 +55,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.util.StringUtils; @@ -217,7 +219,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements } try { try { - if ((data = ZKUtil.getDataNoWatch(watcher, path, stat)) == null) { + if ((data = ZooKeeperUtil.getDataNoWatch(watcher, path, stat)) == null) { SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.increment(); return; } @@ -333,7 +335,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements int availableRSs = 1; try { List regionServers = - ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode); + ZooKeeperUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode); availableRSs = Math.max(availableRSs, (regionServers == null) ? 0 : regionServers.size()); } catch (KeeperException e) { // do nothing @@ -427,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.znodePaths.splitLogZNode, paths.get(idx))); + grabTask(ZooKeeperUtil.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."); @@ -451,10 +453,10 @@ 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.znodePaths.recoveringRegionsZNode, + String nodePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, region); try { - if (ZKUtil.checkExists(watcher, nodePath) == -1) { + if (ZooKeeperUtil.checkExists(watcher, nodePath) == -1) { server.getExecutorService().submit( new FinishRegionRecoveringHandler(server, region, nodePath)); } else { @@ -485,7 +487,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements // it will come out if worker thread exited. while (!shouldStop) { try { - childrenPaths = ZKUtil.listChildrenAndWatchForNewChildren(watcher, + childrenPaths = ZooKeeperUtil.listChildrenAndWatchForNewChildren(watcher, watcher.znodePaths.splitLogZNode); if (childrenPaths != null) { return childrenPaths; @@ -509,7 +511,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements public boolean isReady() throws InterruptedException { int result = -1; try { - result = ZKUtil.checkExists(watcher, watcher.znodePaths.splitLogZNode); + result = ZooKeeperUtil.checkExists(watcher, watcher.znodePaths.splitLogZNode); } catch (KeeperException e) { // ignore LOG.warn("Exception when checking for " + watcher.znodePaths.splitLogZNode @@ -553,7 +555,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements @Override public RegionStoreSequenceIds getRegionFlushedSequenceId(String failedServerName, String key) throws IOException { - return ZKSplitLog.getRegionFlushedSequenceId(watcher, failedServerName, key); + return getRegionFlushedSequenceId(watcher, failedServerName, key); } /** @@ -590,7 +592,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements String task = zkDetails.getTaskNode(); int taskZKVersion = zkDetails.getCurTaskZKVersion().intValue(); try { - if (ZKUtil.setData(watcher, task, slt.toByteArray(), taskZKVersion)) { + if (ZooKeeperUtil.setData(watcher, task, slt.toByteArray(), taskZKVersion)) { LOG.info("successfully transitioned task " + task + " to final state " + slt); ctr.increment(); return; @@ -647,4 +649,44 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements } } + /** + * This function is used in distributedLogReplay to fetch last flushed sequence id from ZK + * @param zkw + * @param serverName + * @param encodedRegionName + * @return the last flushed sequence ids recorded in ZK of the region for serverName + * @throws IOException + */ + + public static RegionStoreSequenceIds getRegionFlushedSequenceId(ZooKeeperWatcher zkw, + String serverName, String encodedRegionName) throws IOException { + // when SplitLogWorker recovers a region by directly replaying unflushed WAL edits, + // last flushed sequence Id changes when newly assigned RS flushes writes to the region. + // If the newly assigned RS fails again(a chained RS failures scenario), the last flushed + // sequence Id name space (sequence Id only valid for a particular RS instance), changes + // when different newly assigned RS flushes the region. + // 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 = ZooKeeperUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName); + nodePath = ZooKeeperUtil.joinZNode(nodePath, serverName); + try { + byte[] data; + try { + data = ZooKeeperUtil.getData(zkw, nodePath); + } catch (InterruptedException e) { + throw new InterruptedIOException(); + } + if (data != null) { + result = ZKUtil.parseRegionStoreSequenceIds(data); + } + } catch (KeeperException e) { + throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server=" + + serverName + "; region=" + encodedRegionName, e); + } catch (DeserializationException e) { + LOG.warn("Can't parse last flushed sequence Id from znode:" + nodePath, e); + } + return result; + } + } 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 25e1ec8c3c..a501f5dbac 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 @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; @@ -31,7 +32,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -122,7 +122,7 @@ public class ActiveMasterManager extends ZooKeeperListener { // Watch the node and check if it exists. try { synchronized(clusterHasActiveMaster) { - if (ZKUtil.watchAndCheckExists(watcher, watcher.znodePaths.masterAddressZNode)) { + if (ZooKeeperUtil.watchAndCheckExists(watcher, watcher.znodePaths.masterAddressZNode)) { // A master node exists, there is an active master LOG.debug("A master is now available"); clusterHasActiveMaster.set(true); @@ -155,7 +155,7 @@ public class ActiveMasterManager extends ZooKeeperListener { */ boolean blockUntilBecomingActiveMaster( int checkInterval, MonitoredTask startupStatus) { - String backupZNode = ZKUtil.joinZNode( + String backupZNode = ZooKeeperUtil.joinZNode( this.watcher.znodePaths.backupMasterAddressesZNode, this.sn.toString()); while (!(master.isAborted() || master.isStopped())) { startupStatus.setStatus("Trying to register in ZK as active master"); @@ -167,9 +167,9 @@ public class ActiveMasterManager extends ZooKeeperListener { // If we were a backup master before, delete our ZNode from the backup // master directory since we are the active now) - if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) { + if (ZooKeeperUtil.checkExists(this.watcher, backupZNode) != -1) { LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory"); - ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode); + ZooKeeperUtil.deleteNodeFailSilent(this.watcher, backupZNode); } // Save the znode in a file, this will allow to check if we crash in the launch scripts ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString()); @@ -187,7 +187,7 @@ public class ActiveMasterManager extends ZooKeeperListener { String msg; byte[] bytes = - ZKUtil.getDataAndWatch(this.watcher, this.watcher.znodePaths.masterAddressZNode); + ZooKeeperUtil.getDataAndWatch(this.watcher, this.watcher.znodePaths.masterAddressZNode); if (bytes == null) { msg = ("A master was detected, but went down before its address " + "could be read. Attempting to become the next active master"); @@ -204,7 +204,7 @@ public class ActiveMasterManager extends ZooKeeperListener { msg = ("Current master has this master's address, " + currentMaster + "; master was restarted? Deleting node."); // Hurry along the expiration of the znode. - ZKUtil.deleteNode(this.watcher, this.watcher.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(this.watcher, this.watcher.znodePaths.masterAddressZNode); // We may have failed to delete the znode at the previous step, but // we delete the file anyway: a second attempt to delete the znode is likely to fail again. @@ -244,7 +244,7 @@ public class ActiveMasterManager extends ZooKeeperListener { */ boolean hasActiveMaster() { try { - if (ZKUtil.checkExists(watcher, watcher.znodePaths.masterAddressZNode) >= 0) { + if (ZooKeeperUtil.checkExists(watcher, watcher.znodePaths.masterAddressZNode) >= 0) { return true; } } @@ -270,7 +270,7 @@ public class ActiveMasterManager extends ZooKeeperListener { LOG.warn("Failed get of master address: " + e.toString()); } if (activeMaster != null && activeMaster.equals(this.sn)) { - ZKUtil.deleteNode(watcher, watcher.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(watcher, watcher.znodePaths.masterAddressZNode); // We may have failed to delete the znode at the previous step, but // we delete the file anyway: a second attempt to delete the znode is likely to fail again. ZNodeClearer.deleteMyEphemeralNodeOnDisk(); 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/master/DrainingServerTracker.java similarity index 88% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java index cdc67942e8..3c5a6e5672 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DrainingServerTracker.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.master; import java.io.IOException; import java.util.List; @@ -24,11 +24,12 @@ import java.util.TreeSet; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.master.ServerListener; -import org.apache.hadoop.hbase.master.ServerManager; import org.apache.zookeeper.KeeperException; /** @@ -82,7 +83,7 @@ public class DrainingServerTracker extends ZooKeeperListener { } }); List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode); + ZooKeeperUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode); add(servers); } @@ -90,7 +91,7 @@ public class DrainingServerTracker extends ZooKeeperListener { synchronized(this.drainingServers) { this.drainingServers.clear(); for (String n: servers) { - final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n)); + final ServerName sn = ServerName.valueOf(ZooKeeperUtil.getNodeName(n)); this.drainingServers.add(sn); this.serverManager.addServerToDrainList(sn); LOG.info("Draining RS node created, adding to list [" + @@ -110,7 +111,7 @@ public class DrainingServerTracker extends ZooKeeperListener { @Override public void nodeDeleted(final String path) { if(path.startsWith(watcher.znodePaths.drainingZNode)) { - final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path)); + final ServerName sn = ServerName.valueOf(ZooKeeperUtil.getNodeName(path)); LOG.info("Draining RS node deleted, removing from list [" + sn + "]"); remove(sn); @@ -122,7 +123,7 @@ public class DrainingServerTracker extends ZooKeeperListener { if(path.equals(watcher.znodePaths.drainingZNode)) { try { final List newNodes = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode); + ZooKeeperUtil.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/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index a990a4bce4..2ebd8fe01c 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 @@ -176,15 +176,12 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.util.ZKDataMigrator; -import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker; import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker; -import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; -import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -868,7 +865,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().znodePaths.tableZNode); + ZooKeeperUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().znodePaths.tableZNode); status.setStatus("Submitting log splitting work for previously failed region servers"); metaBootstrap.processDeadServers(); @@ -1995,7 +1992,7 @@ public class HMaster extends HRegionServer implements MasterServices { } private void startActiveMasterManager(int infoPort) throws KeeperException { - String backupZNode = ZKUtil.joinZNode( + String backupZNode = ZooKeeperUtil.joinZNode( zooKeeper.znodePaths.backupMasterAddressesZNode, serverName.toString()); /* * Add a ZNode for ourselves in the backup master directory since we @@ -2490,7 +2487,7 @@ public class HMaster extends HRegionServer implements MasterServices { // Build Set of backup masters from ZK nodes List backupMasterStrings; try { - backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper, + backupMasterStrings = ZooKeeperUtil.listChildrenNoWatch(this.zooKeeper, this.zooKeeper.znodePaths.backupMasterAddressesZNode); } catch (KeeperException e) { LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e); @@ -2504,7 +2501,7 @@ public class HMaster extends HRegionServer implements MasterServices { try { byte [] bytes; try { - bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode( + bytes = ZooKeeperUtil.getData(this.zooKeeper, ZooKeeperUtil.joinZNode( this.zooKeeper.znodePaths.backupMasterAddressesZNode, s)); } catch (InterruptedException e) { throw new InterruptedIOException(); @@ -3426,8 +3423,8 @@ public class HMaster extends HRegionServer implements MasterServices { String parentZnode = getZooKeeper().znodePaths.drainingZNode; for (ServerName server : servers) { try { - String node = ZKUtil.joinZNode(parentZnode, server.getServerName()); - ZKUtil.createAndFailSilent(getZooKeeper(), node); + String node = ZooKeeperUtil.joinZNode(parentZnode, server.getServerName()); + ZooKeeperUtil.createAndFailSilent(getZooKeeper(), node); } catch (KeeperException ke) { throw new HBaseIOException( this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke); @@ -3473,9 +3470,9 @@ public class HMaster extends HRegionServer implements MasterServices { final List encodedRegionNames) throws HBaseIOException { // Remove the server from decommissioned (draining) server list. String parentZnode = getZooKeeper().znodePaths.drainingZNode; - String node = ZKUtil.joinZNode(parentZnode, server.getServerName()); + String node = ZooKeeperUtil.joinZNode(parentZnode, server.getServerName()); try { - ZKUtil.deleteNodeFailSilent(getZooKeeper(), node); + ZooKeeperUtil.deleteNodeFailSilent(getZooKeeper(), node); } catch (KeeperException ke) { throw new HBaseIOException( this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java index f9a441d5ab..90471b53e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java @@ -33,7 +33,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ZNodeClearer; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.ServerCommandLine; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.zookeeper.KeeperException; @@ -196,7 +196,7 @@ public class HMasterCommandLine extends ServerCommandLine { } // login the zookeeper server principal (if using security) - ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE, + ZooKeeperUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE, HConstants.ZK_SERVER_KERBEROS_PRINCIPAL, null); int localZKClusterSessionTimeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT + ".localHBaseCluster", 10*1000); 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 677a2a0b30..ccf60894b6 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -98,7 +98,7 @@ public class MasterMetaBootstrap { // send a close and wait for a max of 30 seconds ServerManager.closeRegionSilentlyAndWait(master.getClusterConnection(), r.getServerName(), r.getRegion(), 30000); - ZKUtil.deleteNode(zooKeeper, zooKeeper.znodePaths.getZNodeForReplica(replicaId)); + ZooKeeperUtil.deleteNode(zooKeeper, zooKeeper.znodePaths.getZNodeForReplica(replicaId)); } } } catch (Exception ex) { 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/master/RegionServerTracker.java similarity index 88% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java index 47d504c03d..ebaabf62f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.master; import java.io.IOException; import java.io.InterruptedIOException; @@ -28,9 +28,10 @@ import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.master.MasterServices; -import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo; import org.apache.zookeeper.KeeperException; @@ -70,7 +71,7 @@ public class RegionServerTracker extends ZooKeeperListener { public void start() throws KeeperException, IOException { watcher.registerListener(this); List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); + ZooKeeperUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); refresh(servers); } @@ -78,12 +79,12 @@ public class RegionServerTracker extends ZooKeeperListener { synchronized(this.regionServers) { this.regionServers.clear(); for (String n: servers) { - ServerName sn = ServerName.parseServerName(ZKUtil.getNodeName(n)); + ServerName sn = ServerName.parseServerName(ZooKeeperUtil.getNodeName(n)); if (regionServers.get(sn) == null) { RegionServerInfo.Builder rsInfoBuilder = RegionServerInfo.newBuilder(); try { - String nodePath = ZKUtil.joinZNode(watcher.znodePaths.rsZNode, n); - byte[] data = ZKUtil.getData(watcher, nodePath); + String nodePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.rsZNode, n); + byte[] data = ZooKeeperUtil.getData(watcher, nodePath); if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { int magicLen = ProtobufUtil.lengthOfPBMagic(); ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, data.length - magicLen); @@ -116,7 +117,7 @@ public class RegionServerTracker extends ZooKeeperListener { @Override public void nodeDeleted(String path) { if (path.startsWith(watcher.znodePaths.rsZNode)) { - String serverName = ZKUtil.getNodeName(path); + String serverName = ZooKeeperUtil.getNodeName(path); LOG.info("RegionServer ephemeral node deleted, processing expiration [" + serverName + "]"); ServerName sn = ServerName.parseServerName(serverName); @@ -136,7 +137,7 @@ public class RegionServerTracker extends ZooKeeperListener { && !server.isAborted() && !server.isStopped()) { try { List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); + ZooKeeperUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); refresh(servers); } catch (IOException e) { server.abort("Unexpected zk exception getting RS nodes", e); @@ -149,7 +150,7 @@ public class RegionServerTracker extends ZooKeeperListener { public RegionServerInfo getRegionServerInfo(final ServerName sn) { return regionServers.get(sn); } - + /** * Gets the online servers. * @return list of online servers 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 c0143030a4..b847f71065 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 @@ -42,14 +42,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClockOutOfSyncException; -import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.YouAreDeadException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RetriesExhaustedException; @@ -58,7 +57,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -556,7 +555,7 @@ public class ServerManager { private List getRegionServersInZK(final ZooKeeperWatcher zkw) throws KeeperException { - return ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); + return ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java similarity index 84% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java index 8323ec4692..0341f6ca1d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/SplitOrMergeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitOrMergeTracker.java @@ -15,12 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.master; import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -46,15 +49,15 @@ public class SplitOrMergeTracker { public SplitOrMergeTracker(ZooKeeperWatcher watcher, Configuration conf, Abortable abortable) { try { - if (ZKUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) { - ZKUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode); + if (ZooKeeperUtil.checkExists(watcher, watcher.znodePaths.switchZNode) < 0) { + ZooKeeperUtil.createAndFailSilent(watcher, watcher.znodePaths.switchZNode); } } catch (KeeperException e) { throw new RuntimeException(e); } - splitZnode = ZKUtil.joinZNode(watcher.znodePaths.switchZNode, + splitZnode = ZooKeeperUtil.joinZNode(watcher.znodePaths.switchZNode, conf.get("zookeeper.znode.switch.split", "split")); - mergeZnode = ZKUtil.joinZNode(watcher.znodePaths.switchZNode, + mergeZnode = ZooKeeperUtil.joinZNode(watcher.znodePaths.switchZNode, conf.get("zookeeper.znode.switch.merge", "merge")); splitStateTracker = new SwitchStateTracker(watcher, splitZnode, abortable); mergeStateTracker = new SwitchStateTracker(watcher, mergeZnode, abortable); @@ -67,12 +70,12 @@ public class SplitOrMergeTracker { public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) { switch (switchType) { - case SPLIT: - return splitStateTracker.isSwitchEnabled(); - case MERGE: - return mergeStateTracker.isSwitchEnabled(); - default: - break; + case SPLIT: + return splitStateTracker.isSwitchEnabled(); + case MERGE: + return mergeStateTracker.isSwitchEnabled(); + default: + break; } return false; } @@ -120,9 +123,9 @@ public class SplitOrMergeTracker { public void setSwitchEnabled(boolean enabled) throws KeeperException { byte [] upData = toByteArray(enabled); try { - ZKUtil.setData(watcher, node, upData); + ZooKeeperUtil.setData(watcher, node, upData); } catch(KeeperException.NoNodeException nne) { - ZKUtil.createAndWatch(watcher, node, upData); + ZooKeeperUtil.createAndWatch(watcher, node, upData); } super.nodeDataChanged(node); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java index 822ca6f6f3..9923c93600 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java @@ -30,6 +30,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeers; @@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -111,7 +111,7 @@ public class ReplicationZKNodeCleaner { Set peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds()); String hfileRefsZNode = queueDeletor.getHfileRefsZNode(); try { - if (-1 == ZKUtil.checkExists(zkw, hfileRefsZNode)) { + if (-1 == ZooKeeperUtil.checkExists(zkw, hfileRefsZNode)) { return null; } List listOfPeers = this.queuesClient.getAllPeersFromHFileRefsQueue(); @@ -139,12 +139,13 @@ public class ReplicationZKNodeCleaner { * @throws IOException */ public void removeQueue(final String replicator, final String queueId) throws IOException { - String queueZnodePath = ZKUtil.joinZNode(ZKUtil.joinZNode(this.queuesZNode, replicator), + String queueZnodePath = ZooKeeperUtil + .joinZNode(ZooKeeperUtil.joinZNode(this.queuesZNode, replicator), queueId); try { ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) { - ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath); + ZooKeeperUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath); LOG.info("Successfully removed replication queue, replicator: " + replicator + ", queueId: " + queueId); } @@ -159,10 +160,10 @@ public class ReplicationZKNodeCleaner { * @throws IOException */ public void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException { - String node = ZKUtil.joinZNode(this.hfileRefsZNode, hfileRefsQueueId); + String node = ZooKeeperUtil.joinZNode(this.hfileRefsZNode, hfileRefsQueueId); try { if (!replicationPeers.getAllPeerIds().contains(hfileRefsQueueId)) { - ZKUtil.deleteNodeRecursively(this.zookeeper, node); + ZooKeeperUtil.deleteNodeRecursively(this.zookeeper, node); LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path " + hfileRefsZNode); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java index 8d34fe4227..3ed41840a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java @@ -24,10 +24,10 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -76,7 +76,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { String abortNode = zkProc.getAbortZNode(procName); try { // check to see if the abort node already exists - if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) { + if (ZooKeeperUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) { abort(abortNode); } // If we get an abort node watch triggered here, we'll go complete creating the acquired @@ -93,12 +93,12 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { try { // notify all the procedure listeners to look for the acquire node byte[] data = ProtobufUtil.prependPBMagic(info); - ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data); + ZooKeeperUtil.createWithParents(zkProc.getWatcher(), acquire, data); // loop through all the children of the acquire phase and watch for them for (String node : nodeNames) { - String znode = ZKUtil.joinZNode(acquire, node); + String znode = ZooKeeperUtil.joinZNode(acquire, node); LOG.debug("Watching for acquire node:" + znode); - if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) { + if (ZooKeeperUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) { coordinator.memberAcquiredBarrier(procName, node); } } @@ -116,12 +116,12 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { LOG.debug("Creating reached barrier zk node:" + reachedNode); try { // create the reached znode and watch for the reached znodes - ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode); + ZooKeeperUtil.createWithParents(zkProc.getWatcher(), reachedNode); // loop through all the children of the acquire phase and watch for them for (String node : nodeNames) { - String znode = ZKUtil.joinZNode(reachedNode, node); - if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) { - byte[] dataFromMember = ZKUtil.getData(zkProc.getWatcher(), znode); + String znode = ZooKeeperUtil.joinZNode(reachedNode, node); + if (ZooKeeperUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) { + byte[] dataFromMember = ZooKeeperUtil.getData(zkProc.getWatcher(), znode); // ProtobufUtil.isPBMagicPrefix will check null if (dataFromMember != null && dataFromMember.length > 0) { if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) { @@ -196,17 +196,17 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { logZKTree(this.baseZNode); if (isAcquiredPathNode(path)) { // node wasn't present when we created the watch so zk event triggers acquire - coordinator.memberAcquiredBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)), - ZKUtil.getNodeName(path)); + coordinator.memberAcquiredBarrier(ZooKeeperUtil.getNodeName(ZooKeeperUtil.getParent(path)), + ZooKeeperUtil.getNodeName(path)); } else if (isReachedPathNode(path)) { // node was absent when we created the watch so zk event triggers the finished barrier. // TODO Nothing enforces that acquire and reached znodes from showing up in wrong order. - String procName = ZKUtil.getNodeName(ZKUtil.getParent(path)); - String member = ZKUtil.getNodeName(path); + String procName = ZooKeeperUtil.getNodeName(ZooKeeperUtil.getParent(path)); + String member = ZooKeeperUtil.getNodeName(path); // get the data from the procedure member try { - byte[] dataFromMember = ZKUtil.getData(watcher, path); + byte[] dataFromMember = ZooKeeperUtil.getData(watcher, path); // ProtobufUtil.isPBMagicPrefix will check null if (dataFromMember != null && dataFromMember.length > 0) { if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) { @@ -264,7 +264,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { String source = (ee.getSource() == null) ? coordName : ee.getSource(); byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee)); // first create the znode for the procedure - ZKUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo); + ZooKeeperUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo); LOG.debug("Finished creating abort node:" + procAbortNode); } catch (KeeperException e) { // possible that we get this error for the procedure if we already reset the zk state, but in @@ -280,10 +280,10 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs { * @param abortNode full znode path to the failed procedure information */ protected void abort(String abortNode) { - String procName = ZKUtil.getNodeName(abortNode); + String procName = ZooKeeperUtil.getNodeName(abortNode); ForeignException ee = null; try { - byte[] data = ZKUtil.getData(zkProc.getWatcher(), abortNode); + byte[] data = ZooKeeperUtil.getData(zkProc.getWatcher(), abortNode); if (data == null || data.length == 0) { // ignore return; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java index f8db277eee..01076575d7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java @@ -23,11 +23,11 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -83,7 +83,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { watchForAbortedProcedures(); return; } - String parent = ZKUtil.getParent(path); + String parent = ZooKeeperUtil.getParent(path); // if its the end barrier, the procedure can be completed if (isReachedNode(parent)) { receivedReachedGlobalBarrier(path); @@ -126,7 +126,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { */ private void receivedReachedGlobalBarrier(String path) { LOG.debug("Received reached global barrier:" + path); - String procName = ZKUtil.getNodeName(path); + String procName = ZooKeeperUtil.getNodeName(path); this.member.receivedReachedGlobalBarrier(procName); } @@ -134,9 +134,9 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { LOG.debug("Checking for aborted procedures on node: '" + zkController.getAbortZnode() + "'"); try { // this is the list of the currently aborted procedues - for (String node : ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(), + for (String node : ZooKeeperUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(), zkController.getAbortZnode())) { - String abortNode = ZKUtil.joinZNode(zkController.getAbortZnode(), node); + String abortNode = ZooKeeperUtil.joinZNode(zkController.getAbortZnode(), node); abort(abortNode); } } catch (KeeperException e) { @@ -150,7 +150,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { LOG.debug("Looking for new procedures under znode:'" + zkController.getAcquiredBarrier() + "'"); List runningProcedures = null; try { - runningProcedures = ZKUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(), + runningProcedures = ZooKeeperUtil.listChildrenAndWatchForNewChildren(zkController.getWatcher(), zkController.getAcquiredBarrier()); if (runningProcedures == null) { LOG.debug("No running procedures."); @@ -166,7 +166,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { } for (String procName : runningProcedures) { // then read in the procedure information - String path = ZKUtil.joinZNode(zkController.getAcquiredBarrier(), procName); + String path = ZooKeeperUtil.joinZNode(zkController.getAcquiredBarrier(), procName); startNewSubprocedure(path); } } @@ -180,11 +180,11 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { */ private synchronized void startNewSubprocedure(String path) { LOG.debug("Found procedure znode: " + path); - String opName = ZKUtil.getNodeName(path); + String opName = ZooKeeperUtil.getNodeName(path); // start watching for an abort notification for the procedure String abortZNode = zkController.getAbortZNode(opName); try { - if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), abortZNode)) { + if (ZooKeeperUtil.watchAndCheckExists(zkController.getWatcher(), abortZNode)) { LOG.debug("Not starting:" + opName + " because we already have an abort notification."); return; } @@ -197,7 +197,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { // get the data for the procedure Subprocedure subproc = null; try { - byte[] data = ZKUtil.getData(zkController.getWatcher(), path); + byte[] data = ZooKeeperUtil.getData(zkController.getWatcher(), path); if (!ProtobufUtil.isPBMagicPrefix(data)) { String msg = "Data in for starting procedure " + opName + " is illegally formatted (no pb magic). " + @@ -238,14 +238,14 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { try { LOG.debug("Member: '" + memberName + "' joining acquired barrier for procedure (" + procName + ") in zk"); - String acquiredZNode = ZKUtil.joinZNode(ZKProcedureUtil.getAcquireBarrierNode( + String acquiredZNode = ZooKeeperUtil.joinZNode(ZKProcedureUtil.getAcquireBarrierNode( zkController, procName), memberName); - ZKUtil.createAndFailSilent(zkController.getWatcher(), acquiredZNode); + ZooKeeperUtil.createAndFailSilent(zkController.getWatcher(), acquiredZNode); // watch for the complete node for this snapshot String reachedBarrier = zkController.getReachedBarrierNode(procName); LOG.debug("Watch for global barrier reached:" + reachedBarrier); - if (ZKUtil.watchAndCheckExists(zkController.getWatcher(), reachedBarrier)) { + if (ZooKeeperUtil.watchAndCheckExists(zkController.getWatcher(), reachedBarrier)) { receivedReachedGlobalBarrier(reachedBarrier); } } catch (KeeperException e) { @@ -262,13 +262,14 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { String procName = sub.getName(); LOG.debug("Marking procedure '" + procName + "' completed for member '" + memberName + "' in zk"); - String joinPath = ZKUtil.joinZNode(zkController.getReachedBarrierNode(procName), memberName); + String joinPath = ZooKeeperUtil + .joinZNode(zkController.getReachedBarrierNode(procName), memberName); // ProtobufUtil.prependPBMagic does not take care of null if (data == null) { data = new byte[0]; } try { - ZKUtil.createAndFailSilent(zkController.getWatcher(), joinPath, + ZooKeeperUtil.createAndFailSilent(zkController.getWatcher(), joinPath, ProtobufUtil.prependPBMagic(data)); } catch (KeeperException e) { member.controllerConnectionFailure("Failed to post zk node:" + joinPath @@ -292,7 +293,7 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { try { String source = (ee.getSource() == null) ? memberName: ee.getSource(); byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee)); - ZKUtil.createAndFailSilent(zkController.getWatcher(), procAbortZNode, errorInfo); + ZooKeeperUtil.createAndFailSilent(zkController.getWatcher(), procAbortZNode, errorInfo); LOG.debug("Finished creating abort znode:" + procAbortZNode); } catch (KeeperException e) { // possible that we get this error for the procedure if we already reset the zk state, but in @@ -309,9 +310,9 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs { */ protected void abort(String abortZNode) { LOG.debug("Aborting procedure member for znode " + abortZNode); - String opName = ZKUtil.getNodeName(abortZNode); + String opName = ZooKeeperUtil.getNodeName(abortZNode); try { - byte[] data = ZKUtil.getData(zkController.getWatcher(), abortZNode); + byte[] data = ZooKeeperUtil.getData(zkController.getWatcher(), abortZNode); // figure out the data we need to pass ForeignException ee; 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 a19ecb2254..3c73a32f7f 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 @@ -23,8 +23,8 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -78,17 +78,17 @@ 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.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); + this.baseZNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, procDescription); + acquiredZnode = ZooKeeperUtil.joinZNode(baseZNode, ACQUIRED_BARRIER_ZNODE_DEFAULT); + reachedZnode = ZooKeeperUtil.joinZNode(baseZNode, REACHED_BARRIER_ZNODE_DEFAULT); + abortZnode = ZooKeeperUtil.joinZNode(baseZNode, ABORT_ZNODE_DEFAULT); // first make sure all the ZK nodes exist // make sure all the parents exist (sometimes not the case in tests) - ZKUtil.createWithParents(watcher, acquiredZnode); + ZooKeeperUtil.createWithParents(watcher, acquiredZnode); // regular create because all the parents exist - ZKUtil.createAndFailSilent(watcher, reachedZnode); - ZKUtil.createAndFailSilent(watcher, abortZnode); + ZooKeeperUtil.createAndFailSilent(watcher, reachedZnode); + ZooKeeperUtil.createAndFailSilent(watcher, abortZnode); } @Override @@ -130,7 +130,7 @@ public abstract class ZKProcedureUtil */ public static String getAcquireBarrierNode(ZKProcedureUtil controller, String opInstanceName) { - return ZKUtil.joinZNode(controller.acquiredZnode, opInstanceName); + return ZooKeeperUtil.joinZNode(controller.acquiredZnode, opInstanceName); } /** @@ -142,7 +142,7 @@ public abstract class ZKProcedureUtil */ public static String getReachedBarrierNode(ZKProcedureUtil controller, String opInstanceName) { - return ZKUtil.joinZNode(controller.reachedZnode, opInstanceName); + return ZooKeeperUtil.joinZNode(controller.reachedZnode, opInstanceName); } /** @@ -153,7 +153,7 @@ public abstract class ZKProcedureUtil * @return full znode path to the abort znode */ public static String getAbortNode(ZKProcedureUtil controller, String opInstanceName) { - return ZKUtil.joinZNode(controller.abortZnode, opInstanceName); + return ZooKeeperUtil.joinZNode(controller.abortZnode, opInstanceName); } public ZooKeeperWatcher getWatcher() { @@ -212,7 +212,7 @@ public abstract class ZKProcedureUtil private boolean isMemberNode(final String path, final String statePath) { int count = 0; for (int i = statePath.length(); i < path.length(); ++i) { - count += (path.charAt(i) == ZKUtil.ZNODE_PATH_SEPARATOR) ? 1 : 0; + count += (path.charAt(i) == ZooKeeperUtil.ZNODE_PATH_SEPARATOR) ? 1 : 0; } return count == 2; } @@ -257,11 +257,11 @@ public abstract class ZKProcedureUtil * @throws KeeperException if an unexpected exception occurs */ protected void logZKTree(String root, String prefix) throws KeeperException { - List children = ZKUtil.listChildrenNoWatch(watcher, root); + List children = ZooKeeperUtil.listChildrenNoWatch(watcher, root); if (children == null) return; for (String child : children) { LOG.debug(prefix + child); - String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child); + String node = ZooKeeperUtil.joinZNode(root.equals("/") ? "" : root, child); logZKTree(node, prefix + "---"); } } @@ -273,7 +273,8 @@ public abstract class ZKProcedureUtil // If the coordinator was shutdown mid-procedure, then we are going to lose // an procedure that was previously started by cleaning out all the previous state. Its much // harder to figure out how to keep an procedure going and the subject of HBASE-5487. - ZKUtil.deleteChildrenRecursivelyMultiOrSequential(watcher, true, acquiredZnode, reachedZnode, + ZooKeeperUtil + .deleteChildrenRecursivelyMultiOrSequential(watcher, true, acquiredZnode, reachedZnode, abortZnode); if (LOG.isTraceEnabled()) { @@ -290,10 +291,10 @@ public abstract class ZKProcedureUtil String reachedBarrierNode = getReachedBarrierNode(procedureName); String abortZNode = getAbortZNode(procedureName); - ZKUtil.createAndFailSilent(watcher, acquiredBarrierNode); - ZKUtil.createAndFailSilent(watcher, abortZNode); + ZooKeeperUtil.createAndFailSilent(watcher, acquiredBarrierNode); + ZooKeeperUtil.createAndFailSilent(watcher, abortZNode); - ZKUtil.deleteNodeRecursivelyMultiOrSequential(watcher, true, acquiredBarrierNode, + ZooKeeperUtil.deleteNodeRecursivelyMultiOrSequential(watcher, true, acquiredBarrierNode, reachedBarrierNode, abortZNode); if (LOG.isTraceEnabled()) { 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 f384c1f5aa..fb6ce09183 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 @@ -160,11 +160,10 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.metrics2.util.MBeans; @@ -601,7 +600,7 @@ public class HRegionServer extends HasThread implements rpcRetryingCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf); // login the zookeeper client principal (if using security) - ZKUtil.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE, + ZooKeeperUtil.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE, HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL, hostName); // login the server principal (if using secure Hadoop) login(userProvider, hostName); @@ -1598,11 +1597,11 @@ public class HRegionServer extends HasThread implements rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1); rsInfo.setVersionInfo(ProtobufUtil.getVersionInfo()); byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray()); - ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(), data); + ZooKeeperUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(), data); } private void deleteMyEphemeralNode() throws KeeperException { - ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath()); + ZooKeeperUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath()); } @Override @@ -3548,7 +3547,7 @@ public class HRegionServer extends HasThread implements } private String getMyEphemeralNodePath() { - return ZKUtil.joinZNode(this.zooKeeper.znodePaths.rsZNode, getServerName().toString()); + return ZooKeeperUtil.joinZNode(this.zooKeeper.znodePaths.rsZNode, getServerName().toString()); } private boolean isHealthCheckerConfigured() { @@ -3590,25 +3589,25 @@ public class HRegionServer extends HasThread implements try { long lastRecordedFlushedSequenceId = -1; - String nodePath = ZKUtil.joinZNode(this.zooKeeper.znodePaths.recoveringRegionsZNode, + String nodePath = ZooKeeperUtil.joinZNode(this.zooKeeper.znodePaths.recoveringRegionsZNode, regionInfo.getEncodedName()); // recovering-region level byte[] data; try { - data = ZKUtil.getData(zkw, nodePath); + data = ZooKeeperUtil.getData(zkw, nodePath); } catch (InterruptedException e) { throw new InterruptedIOException(); } if (data != null) { - lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data); + lastRecordedFlushedSequenceId = ZKUtil.parseLastFlushedSequenceIdFrom(data); } if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) { - ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay)); + ZooKeeperUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay)); } if (previousRSName != null) { // one level deeper for the failed RS - nodePath = ZKUtil.joinZNode(nodePath, previousRSName); - ZKUtil.setData(zkw, nodePath, + nodePath = ZooKeeperUtil.joinZNode(nodePath, previousRSName); + ZooKeeperUtil.setData(zkw, nodePath, ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores)); LOG.debug("Update last flushed sequence id of region " + regionInfo.getEncodedName() + " for " + previousRSName); @@ -3631,15 +3630,16 @@ public class HRegionServer extends HasThread implements String result = null; long maxZxid = 0; ZooKeeperWatcher zkw = this.getZooKeeper(); - String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName); - List failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath); + String nodePath = ZooKeeperUtil + .joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName); + List failedServers = ZooKeeperUtil.listChildrenNoWatch(zkw, nodePath); if (failedServers == null || failedServers.isEmpty()) { return result; } for (String failedServer : failedServers) { - String rsPath = ZKUtil.joinZNode(nodePath, failedServer); + String rsPath = ZooKeeperUtil.joinZNode(nodePath, failedServer); Stat stat = new Stat(); - ZKUtil.getDataNoWatch(zkw, rsPath, stat); + ZooKeeperUtil.getDataNoWatch(zkw, rsPath, stat); if (maxZxid < stat.getCzxid()) { maxZxid = stat.getCzxid(); result = failedServer; 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/regionserver/RecoveringRegionWatcher.java similarity index 91% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RecoveringRegionWatcher.java index 16485ee476..a855e6f230 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RecoveringRegionWatcher.java @@ -16,12 +16,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.zookeeper; +package org.apache.hadoop.hbase.regionserver; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler; import org.apache.zookeeper.KeeperException; @@ -84,7 +86,7 @@ public class RecoveringRegionWatcher extends ZooKeeperListener { } try { - ZKUtil.getDataAndWatch(watcher, path); + ZooKeeperUtil.getDataAndWatch(watcher, path); } catch (KeeperException e) { LOG.warn("Can't register watcher on znode " + path, e); } 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 a34bedd5ba..af0150839a 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 @@ -26,11 +26,11 @@ import java.util.UUID; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -157,7 +157,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint */ protected static List fetchSlavesAddresses(ZooKeeperWatcher zkw) throws KeeperException { - List children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode); + List children = ZooKeeperUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode); if (children == null) { return Collections.emptyList(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java index a14bd0101d..d8998ef351 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java index 0585c97c3f..d1779c1f41 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper; @@ -30,7 +31,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -55,7 +55,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase { public void update() { List znodes = null; try { - znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); + znodes = ZooKeeperUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); } catch (KeeperException e) { LOG.error("Failed to get peers znode", e); } @@ -71,7 +71,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase { public boolean update(String peerId) { String tableCFsNode = getTableCFsNode(peerId); try { - if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) { + if (ZooKeeperUtil.checkExists(zookeeper, tableCFsNode) != -1) { String peerNode = getPeerNode(peerId); ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode); // We only need to copy data from tableCFs node to rpc Node the first time hmaster start. @@ -80,10 +80,10 @@ public class TableCFsUpdater extends ReplicationStateZKBase { LOG.info("copy tableCFs into peerNode:" + peerId); ReplicationProtos.TableCF[] tableCFs = ReplicationSerDeHelper.parseTableCFs( - ZKUtil.getData(this.zookeeper, tableCFsNode)); + ZooKeeperUtil.getData(this.zookeeper, tableCFsNode)); if (tableCFs != null && tableCFs.length > 0) { rpc.setTableCFsMap(ReplicationSerDeHelper.convert2Map(tableCFs)); - ZKUtil.setData(this.zookeeper, peerNode, + ZooKeeperUtil.setData(this.zookeeper, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); } } else { @@ -106,7 +106,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase { private ReplicationPeerConfig getReplicationPeerConig(String peerNode) throws KeeperException, InterruptedException { byte[] data = null; - data = ZKUtil.getData(this.zookeeper, peerNode); + data = ZooKeeperUtil.getData(this.zookeeper, peerNode); if (data == null) { LOG.error("Could not get configuration for " + "peer because it doesn't exist. peer=" + peerNode); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java index 6c2eb60d14..af5907e39a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.security.access; import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; @@ -746,7 +745,7 @@ public class AccessControlLists { // Deserialize the table permissions from the KV // TODO: This can be improved. Don't build UsersAndPermissions just to unpack it again, // use the builder - AccessControlProtos.UsersAndPermissions.Builder builder = + AccessControlProtos.UsersAndPermissions.Builder builder = AccessControlProtos.UsersAndPermissions.newBuilder(); if (tag.hasArray()) { ProtobufUtil.mergeFrom(builder, tag.getValueArray(), tag.getValueOffset(), tag.getValueLength()); 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 09a1771eff..a8de9e77e6 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 @@ -20,12 +20,12 @@ package org.apache.hadoop.hbase.security.access; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DaemonThreadFactory; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -40,7 +40,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.atomic.AtomicReference; /** * Handles synchronization of access control list entries and updates @@ -67,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.znodePaths.baseZNode, aclZnodeParent); + this.aclZNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, aclZnodeParent); executor = Executors.newSingleThreadExecutor( new DaemonThreadFactory("zk-permission-watcher")); } @@ -75,13 +74,13 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable public void start() throws KeeperException { try { watcher.registerListener(this); - if (ZKUtil.watchAndCheckExists(watcher, aclZNode)) { + if (ZooKeeperUtil.watchAndCheckExists(watcher, aclZNode)) { try { executor.submit(new Callable() { @Override public Void call() throws KeeperException { - List existing = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); + List existing = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); if (existing != null) { refreshNodes(existing); } @@ -125,8 +124,8 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable @Override public void run() { try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); refreshNodes(nodes); } catch (KeeperException ke) { LOG.error("Error reading data from zookeeper", ke); @@ -141,11 +140,11 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable @Override public void nodeDeleted(final String path) { waitUntilStarted(); - if (aclZNode.equals(ZKUtil.getParent(path))) { + if (aclZNode.equals(ZooKeeperUtil.getParent(path))) { asyncProcessNodeUpdate(new Runnable() { @Override public void run() { - String table = ZKUtil.getNodeName(path); + String table = ZooKeeperUtil.getNodeName(path); if(AccessControlLists.isNamespaceEntry(table)) { authManager.removeNamespace(Bytes.toBytes(table)); } else { @@ -159,14 +158,14 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable @Override public void nodeDataChanged(final String path) { waitUntilStarted(); - if (aclZNode.equals(ZKUtil.getParent(path))) { + if (aclZNode.equals(ZooKeeperUtil.getParent(path))) { asyncProcessNodeUpdate(new Runnable() { @Override public void run() { // update cache on an existing table node - String entry = ZKUtil.getNodeName(path); + String entry = ZooKeeperUtil.getNodeName(path); try { - byte[] data = ZKUtil.getDataAndWatch(watcher, path); + byte[] data = ZooKeeperUtil.getDataAndWatch(watcher, path); refreshAuthManager(entry, data); } catch (KeeperException ke) { LOG.error("Error reading data from zookeeper for node " + entry, ke); @@ -186,8 +185,8 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable waitUntilStarted(); if (path.equals(aclZNode)) { try { - final List nodeList = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); + final List nodeList = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode); // preempt any existing nodeChildrenChanged event processing if (childrenChangedFuture != null && !childrenChangedFuture.isDone()) { boolean cancelled = childrenChangedFuture.cancel(true); @@ -222,15 +221,15 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable return null; // No task launched so there will be nothing to cancel later } - private void refreshNodes(List nodes) { - for (ZKUtil.NodeAndData n : nodes) { + private void refreshNodes(List nodes) { + for (ZooKeeperUtil.NodeAndData n : nodes) { if (Thread.interrupted()) { // Use Thread.interrupted so that we clear interrupt status break; } if (n.isEmpty()) continue; String path = n.getNode(); - String entry = (ZKUtil.getNodeName(path)); + String entry = (ZooKeeperUtil.getNodeName(path)); try { refreshAuthManager(entry, n.getData()); } catch (IOException ioe) { @@ -260,12 +259,12 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable */ public void writeToZookeeper(byte[] entry, byte[] permsData) { String entryName = Bytes.toString(entry); - String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); - zkNode = ZKUtil.joinZNode(zkNode, entryName); + String zkNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); + zkNode = ZooKeeperUtil.joinZNode(zkNode, entryName); try { - ZKUtil.createWithParents(watcher, zkNode); - ZKUtil.updateExistingNodeData(watcher, zkNode, permsData, -1); + ZooKeeperUtil.createWithParents(watcher, zkNode); + ZooKeeperUtil.updateExistingNodeData(watcher, zkNode, permsData, -1); } catch (KeeperException e) { LOG.error("Failed updating permissions for entry '" + entryName + "'", e); @@ -278,11 +277,11 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable * @param tableName */ public void deleteTableACLNode(final TableName tableName) { - String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); - zkNode = ZKUtil.joinZNode(zkNode, tableName.getNameAsString()); + String zkNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); + zkNode = ZooKeeperUtil.joinZNode(zkNode, tableName.getNameAsString()); try { - ZKUtil.deleteNode(watcher, zkNode); + ZooKeeperUtil.deleteNode(watcher, zkNode); } catch (KeeperException.NoNodeException e) { LOG.warn("No acl notify node of table '" + tableName + "'"); } catch (KeeperException e) { @@ -295,11 +294,11 @@ 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.znodePaths.baseZNode, ACL_NODE); - zkNode = ZKUtil.joinZNode(zkNode, AccessControlLists.NAMESPACE_PREFIX + namespace); + String zkNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); + zkNode = ZooKeeperUtil.joinZNode(zkNode, AccessControlLists.NAMESPACE_PREFIX + namespace); try { - ZKUtil.deleteNode(watcher, zkNode); + ZooKeeperUtil.deleteNode(watcher, zkNode); } catch (KeeperException.NoNodeException e) { LOG.warn("No acl notify node of namespace '" + namespace + "'"); } catch (KeeperException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java index 718e8e0dc1..98d8c1e275 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Stoppable; @@ -34,7 +35,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKLeaderManager; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.SecretManager; @@ -254,7 +254,7 @@ public class AuthenticationTokenSecretManager } } } - + synchronized boolean isCurrentKeyRolled() { return currentKey != null; } @@ -301,7 +301,7 @@ public class AuthenticationTokenSecretManager setDaemon(true); setName("ZKSecretWatcher-leaderElector"); zkLeader = new ZKLeaderManager(watcher, - ZKUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"), + ZooKeeperUtil.joinZNode(zkWatcher.getRootKeyZNode(), "keymaster"), Bytes.toBytes(serverName), this); } 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 42dc3a9643..829a8ac2dd 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 @@ -25,10 +25,10 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -52,18 +52,18 @@ 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.znodePaths.baseZNode, keyZNodeParent); - this.keysParentZNode = ZKUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT); + this.baseKeyZNode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, keyZNodeParent); + this.keysParentZNode = ZooKeeperUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT); } public void start() throws KeeperException { watcher.registerListener(this); // make sure the base node exists - ZKUtil.createWithParents(watcher, keysParentZNode); + ZooKeeperUtil.createWithParents(watcher, keysParentZNode); - if (ZKUtil.watchAndCheckExists(watcher, keysParentZNode)) { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); + if (ZooKeeperUtil.watchAndCheckExists(watcher, keysParentZNode)) { + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); refreshNodes(nodes); } } @@ -72,8 +72,8 @@ public class ZKSecretWatcher extends ZooKeeperListener { public void nodeCreated(String path) { if (path.equals(keysParentZNode)) { try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); refreshNodes(nodes); } catch (KeeperException ke) { LOG.fatal("Error reading data from zookeeper", ke); @@ -84,8 +84,8 @@ public class ZKSecretWatcher extends ZooKeeperListener { @Override public void nodeDeleted(String path) { - if (keysParentZNode.equals(ZKUtil.getParent(path))) { - String keyId = ZKUtil.getNodeName(path); + if (keysParentZNode.equals(ZooKeeperUtil.getParent(path))) { + String keyId = ZooKeeperUtil.getNodeName(path); try { Integer id = Integer.valueOf(keyId); secretManager.removeKey(id); @@ -97,9 +97,9 @@ public class ZKSecretWatcher extends ZooKeeperListener { @Override public void nodeDataChanged(String path) { - if (keysParentZNode.equals(ZKUtil.getParent(path))) { + if (keysParentZNode.equals(ZooKeeperUtil.getParent(path))) { try { - byte[] data = ZKUtil.getDataAndWatch(watcher, path); + byte[] data = ZooKeeperUtil.getDataAndWatch(watcher, path); if (data == null || data.length == 0) { LOG.debug("Ignoring empty node "+path); return; @@ -123,8 +123,8 @@ public class ZKSecretWatcher extends ZooKeeperListener { if (path.equals(keysParentZNode)) { // keys changed try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); refreshNodes(nodes); } catch (KeeperException ke) { LOG.fatal("Error reading data from zookeeper", ke); @@ -137,10 +137,10 @@ public class ZKSecretWatcher extends ZooKeeperListener { return baseKeyZNode; } - private void refreshNodes(List nodes) { - for (ZKUtil.NodeAndData n : nodes) { + private void refreshNodes(List nodes) { + for (ZooKeeperUtil.NodeAndData n : nodes) { String path = n.getNode(); - String keyId = ZKUtil.getNodeName(path); + String keyId = ZooKeeperUtil.getNodeName(path); try { byte[] data = n.getData(); if (data == null || data.length == 0) { @@ -159,13 +159,13 @@ public class ZKSecretWatcher extends ZooKeeperListener { } private String getKeyNode(int keyId) { - return ZKUtil.joinZNode(keysParentZNode, Integer.toString(keyId)); + return ZooKeeperUtil.joinZNode(keysParentZNode, Integer.toString(keyId)); } public void removeKeyFromZK(AuthenticationKey key) { String keyZNode = getKeyNode(key.getKeyId()); try { - ZKUtil.deleteNode(watcher, keyZNode); + ZooKeeperUtil.deleteNode(watcher, keyZNode); } catch (KeeperException.NoNodeException nne) { LOG.error("Non-existent znode "+keyZNode+" for key "+key.getKeyId(), nne); } catch (KeeperException ke) { @@ -181,7 +181,7 @@ public class ZKSecretWatcher extends ZooKeeperListener { try { byte[] keyData = Writables.getBytes(key); // TODO: is there any point in retrying beyond what ZK client does? - ZKUtil.createSetData(watcher, keyZNode, keyData); + ZooKeeperUtil.createSetData(watcher, keyZNode, keyData); } catch (KeeperException ke) { LOG.fatal("Unable to synchronize master key "+key.getKeyId()+ " to znode "+keyZNode, ke); @@ -198,10 +198,10 @@ public class ZKSecretWatcher extends ZooKeeperListener { try { byte[] keyData = Writables.getBytes(key); try { - ZKUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1); + ZooKeeperUtil.updateExistingNodeData(watcher, keyZNode, keyData, -1); } catch (KeeperException.NoNodeException ne) { // node was somehow removed, try adding it back - ZKUtil.createSetData(watcher, keyZNode, keyData); + ZooKeeperUtil.createSetData(watcher, keyZNode, keyData); } } catch (KeeperException ke) { LOG.fatal("Unable to update master key "+key.getKeyId()+ @@ -213,21 +213,21 @@ public class ZKSecretWatcher extends ZooKeeperListener { watcher.abort("Failed serializing key "+key.getKeyId(), ioe); } } - + /** * refresh keys */ synchronized void refreshKeys() { try { - List nodes = - ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); + List nodes = + ZooKeeperUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode); refreshNodes(nodes); } catch (KeeperException ke) { LOG.fatal("Error reading data from zookeeper", ke); watcher.abort("Error reading changed keys from zookeeper", ke); } } - + /** * get token keys parent node * @return token keys parent node diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java index 964c0f78fa..5ac1ef36ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java @@ -122,7 +122,7 @@ public class VisibilityUtils { /** * Reads back from the zookeeper. The data read here is of the form written by * writeToZooKeeper(Map<byte[], Integer> entries). - * + * * @param data * @return Labels and their ordinal details * @throws DeserializationException @@ -148,7 +148,7 @@ public class VisibilityUtils { * @return User auth details * @throws DeserializationException */ - public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data) + public static MultiUserAuthorizations readUserAuthsFromZKData(byte[] data) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(data)) { int pblen = ProtobufUtil.lengthOfPBMagic(); 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 9b5a44a67d..a64c34cb68 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 @@ -21,9 +21,9 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -38,7 +38,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { private static final Log LOG = LogFactory.getLog(ZKVisibilityLabelWatcher.class); private static final String VISIBILITY_LABEL_ZK_PATH = "zookeeper.znode.visibility.label.parent"; private static final String DEFAULT_VISIBILITY_LABEL_NODE = "visibility/labels"; - private static final String VISIBILITY_USER_AUTHS_ZK_PATH = + private static final String VISIBILITY_USER_AUTHS_ZK_PATH = "zookeeper.znode.visibility.user.auths.parent"; private static final String DEFAULT_VISIBILITY_USER_AUTHS_NODE = "visibility/user_auths"; @@ -53,19 +53,19 @@ 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.znodePaths.baseZNode, labelZnodeParent); - this.userAuthsZnode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, userAuthsZnodeParent); + this.labelZnode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, labelZnodeParent); + this.userAuthsZnode = ZooKeeperUtil.joinZNode(watcher.znodePaths.baseZNode, userAuthsZnodeParent); } public void start() throws KeeperException { watcher.registerListener(this); - ZKUtil.createWithParents(watcher, labelZnode); - ZKUtil.createWithParents(watcher, userAuthsZnode); - byte[] data = ZKUtil.getDataAndWatch(watcher, labelZnode); + ZooKeeperUtil.createWithParents(watcher, labelZnode); + ZooKeeperUtil.createWithParents(watcher, userAuthsZnode); + byte[] data = ZooKeeperUtil.getDataAndWatch(watcher, labelZnode); if (data != null && data.length > 0) { refreshVisibilityLabelsCache(data); } - data = ZKUtil.getDataAndWatch(watcher, userAuthsZnode); + data = ZooKeeperUtil.getDataAndWatch(watcher, userAuthsZnode); if (data != null && data.length > 0) { refreshUserAuthsCache(data); } @@ -91,7 +91,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { public void nodeCreated(String path) { if (path.equals(labelZnode) || path.equals(userAuthsZnode)) { try { - ZKUtil.watchAndCheckExists(watcher, path); + ZooKeeperUtil.watchAndCheckExists(watcher, path); } catch (KeeperException ke) { LOG.error("Error setting watcher on node " + path, ke); // only option is to abort @@ -110,7 +110,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { if (path.equals(labelZnode) || path.equals(userAuthsZnode)) { try { watcher.sync(path); - byte[] data = ZKUtil.getDataAndWatch(watcher, path); + byte[] data = ZooKeeperUtil.getDataAndWatch(watcher, path); if (path.equals(labelZnode)) { refreshVisibilityLabelsCache(data); } else { @@ -131,7 +131,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { /** * Write a labels mirror or user auths mirror into zookeeper - * + * * @param data * @param labelsOrUserAuths true for writing labels and false for user auths. */ @@ -141,7 +141,7 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { znode = this.userAuthsZnode; } try { - ZKUtil.updateExistingNodeData(watcher, znode, data, -1); + ZooKeeperUtil.updateExistingNodeData(watcher, znode, data, -1); } catch (KeeperException e) { LOG.error("Failed writing to " + znode, e); watcher.abort("Failed writing node " + znode + " to zookeeper", e); 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 934a630a67..d7c99ae163 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 @@ -87,7 +87,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -129,7 +129,7 @@ import org.apache.hadoop.hbase.wal.WAL; 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.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.ipc.RemoteException; @@ -703,12 +703,12 @@ public class HBaseFsck extends Configured implements Closeable { */ private boolean setMasterInMaintenanceMode() throws IOException { RetryCounter retryCounter = createZNodeRetryCounterFactory.create(); - hbckEphemeralNodePath = ZKUtil.joinZNode( + hbckEphemeralNodePath = ZooKeeperUtil.joinZNode( zkw.znodePaths.masterMaintZNode, "hbck-" + Long.toString(EnvironmentEdgeManager.currentTime())); do { try { - hbckZodeCreated = ZKUtil.createEphemeralNodeAndWatch(zkw, hbckEphemeralNodePath, null); + hbckZodeCreated = ZooKeeperUtil.createEphemeralNodeAndWatch(zkw, hbckEphemeralNodePath, null); if (hbckZodeCreated) { break; } @@ -735,7 +735,7 @@ public class HBaseFsck extends Configured implements Closeable { private void cleanupHbckZnode() { try { if (zkw != null && hbckZodeCreated) { - ZKUtil.deleteNode(zkw, hbckEphemeralNodePath); + ZooKeeperUtil.deleteNode(zkw, hbckEphemeralNodePath); hbckZodeCreated = false; } } catch (KeeperException e) { @@ -3645,7 +3645,7 @@ public class HBaseFsck extends Configured implements Closeable { private void unassignMetaReplica(HbckInfo hi) throws IOException, InterruptedException, KeeperException { undeployRegions(hi); - ZKUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(hi.metaEntry.getReplicaId())); + ZooKeeperUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(hi.metaEntry.getReplicaId())); } private void assignMetaReplica(int replicaId) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index afb6c5b0e1..7cdd6bc7f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.ClusterStatus.Option; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; 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 43c3598755..345a0af961 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 @@ -25,12 +25,12 @@ import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; @@ -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.znodePaths.tableZNode); + List children = ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.tableZNode); if (children == null) return rv; for (String child: children) { @@ -94,8 +94,8 @@ public class ZKDataMigrator { private static ZooKeeperProtos.DeprecatedTableState.State getTableState( final ZooKeeperWatcher zkw, final TableName tableName) throws KeeperException, InterruptedException { - String znode = ZKUtil.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString()); - byte [] data = ZKUtil.getData(zkw, znode); + String znode = ZooKeeperUtil.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString()); + byte [] data = ZooKeeperUtil.getData(zkw, znode); if (data == null || data.length <= 0) return null; try { ProtobufUtil.expectPBMagicPrefix(data); @@ -109,7 +109,7 @@ public class ZKDataMigrator { ke.initCause(e); throw ke; } catch (DeserializationException e) { - throw ZKUtil.convert(e); + throw ZooKeeperUtil.convert(e); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 1a85997e58..70124b8fde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.ImmutableHRegionInfo; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.BufferedMutator; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java index 12605519f5..27fb566e0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -59,7 +60,7 @@ public class TestMultiVersions { private static final Log LOG = LogFactory.getLog(TestMultiVersions.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private Admin admin; - + private static final int NUM_SLAVES = 3; @Rule @@ -85,7 +86,7 @@ public class TestMultiVersions { * Tests user specifiable time stamps putting, getting and scanning. Also * tests same in presence of deletes. Test cores are written so can be * run against an HRegion and against an HTable: i.e. both local and remote. - * + * *

Port of old TestTimestamp test to here so can better utilize the spun * up cluster running more than a single test per spin up. Keep old tests' * crazyness. @@ -184,7 +185,7 @@ public class TestMultiVersions { * Port of old TestScanMultipleVersions test here so can better utilize the * spun up cluster running more than just a single test. Keep old tests * crazyness. - * + * *

Tests five cases of scans and timestamps. * @throws Exception */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index 4ed8d9173b..4450570ae0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.EmptyWatcher; import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -126,7 +126,7 @@ public class TestZooKeeper { TEST_UTIL.shutdownMiniHBaseCluster(); } finally { TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true); - ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase"); + ZooKeeperUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase"); } } @@ -308,15 +308,15 @@ public class TestZooKeeper { new ZooKeeperWatcher(new Configuration(TEST_UTIL.getConfiguration()), TestZooKeeper.class.getName(), null); byte[] expectedData = new byte[] { 1, 2, 3 }; - ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4/testCreateWithParents", expectedData); - byte[] data = ZKUtil.getData(zkw, "/l1/l2/l3/l4/testCreateWithParents"); + ZooKeeperUtil.createWithParents(zkw, "/l1/l2/l3/l4/testCreateWithParents", expectedData); + byte[] data = ZooKeeperUtil.getData(zkw, "/l1/l2/l3/l4/testCreateWithParents"); assertTrue(Bytes.equals(expectedData, data)); - ZKUtil.deleteNodeRecursively(zkw, "/l1"); + ZooKeeperUtil.deleteNodeRecursively(zkw, "/l1"); - ZKUtil.createWithParents(zkw, "/testCreateWithParents", expectedData); - data = ZKUtil.getData(zkw, "/testCreateWithParents"); + ZooKeeperUtil.createWithParents(zkw, "/testCreateWithParents", expectedData); + data = ZooKeeperUtil.getData(zkw, "/testCreateWithParents"); assertTrue(Bytes.equals(expectedData, data)); - ZKUtil.deleteNodeRecursively(zkw, "/testCreateWithParents"); + ZooKeeperUtil.deleteNodeRecursively(zkw, "/testCreateWithParents"); } /** @@ -329,22 +329,22 @@ public class TestZooKeeper { ZooKeeperWatcher zkw = new ZooKeeperWatcher( new Configuration(TEST_UTIL.getConfiguration()), TestZooKeeper.class.getName(), null); - ZKUtil.createWithParents(zkw, "/l1/l2/l3/l4"); + ZooKeeperUtil.createWithParents(zkw, "/l1/l2/l3/l4"); try { - ZKUtil.deleteNode(zkw, "/l1/l2"); + ZooKeeperUtil.deleteNode(zkw, "/l1/l2"); fail("We should not be able to delete if znode has childs"); } catch (KeeperException ex) { - assertNotNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null)); + assertNotNull(ZooKeeperUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null)); } - ZKUtil.deleteNodeRecursively(zkw, "/l1/l2"); + ZooKeeperUtil.deleteNodeRecursively(zkw, "/l1/l2"); // make sure it really is deleted - assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null)); + assertNull(ZooKeeperUtil.getDataNoWatch(zkw, "/l1/l2/l3/l4", null)); // do the same delete again and make sure it doesn't crash - ZKUtil.deleteNodeRecursively(zkw, "/l1/l2"); + ZooKeeperUtil.deleteNodeRecursively(zkw, "/l1/l2"); - ZKUtil.deleteNode(zkw, "/l1"); - assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2", null)); + ZooKeeperUtil.deleteNode(zkw, "/l1"); + assertNull(ZooKeeperUtil.getDataNoWatch(zkw, "/l1/l2", null)); } /** @@ -430,7 +430,7 @@ public class TestZooKeeper { } } zk.close(); - ZKUtil.createAndFailSilent(zk2, aclZnode); + ZooKeeperUtil.createAndFailSilent(zk2, aclZnode); // Restore the ACL ZooKeeper zk3 = new ZooKeeper(quorumServers, sessionTimeout, EmptyWatcher.instance); @@ -451,7 +451,7 @@ public class TestZooKeeper { public void testGetChildDataAndWatchForNewChildrenShouldNotThrowNPE() throws Exception { ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null); - ZKUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode"); + ZooKeeperUtil.getChildDataAndWatchForNewChildren(zkw, "/wrongNode"); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 20cb5133bf..697635fb94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.StoppableImplementation; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -95,7 +95,7 @@ public class TestZooKeeperTableArchiveClient { // make hfile archiving node so we can archive files ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher(); String archivingZNode = ZKTableArchiveClient.getArchiveZNode(UTIL.getConfiguration(), watcher); - ZKUtil.createWithParents(watcher, archivingZNode); + ZooKeeperUtil.createWithParents(watcher, archivingZNode); rss = mock(RegionServerServices.class); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index d7d1b3ac6e..7a6a7241cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -24,10 +24,9 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 953fae0876..b02d7c943e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -25,15 +25,10 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -54,13 +49,12 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -70,7 +64,6 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java index fb9fb37a90..f7436fe035 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest; @@ -77,7 +77,7 @@ public class TestHBaseAdminNoCluster { * @throws ZooKeeperConnectionException * @throws MasterNotRunningException * @throws ServiceException - * @throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException + * @throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException */ //TODO: Clean up, with Procedure V2 and nonce to prevent the same procedure to call mulitple // time, this test is invalid anymore. Just keep the test around for some time before @@ -85,7 +85,7 @@ public class TestHBaseAdminNoCluster { @Ignore @Test public void testMasterMonitorCallableRetries() - throws MasterNotRunningException, ZooKeeperConnectionException, IOException, + throws MasterNotRunningException, ZooKeeperConnectionException, IOException, org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException { Configuration configuration = HBaseConfiguration.create(); // Set the pause and retry count way down. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java index 98d864bd96..16c67d5695 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; import org.apache.hadoop.hbase.util.HBaseFsckRepair; import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -135,18 +135,18 @@ public class TestMetaWithReplicas { Configuration conf = TEST_UTIL.getConfiguration(); String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String primaryMetaZnode = ZKUtil.joinZNode(baseZNode, + String primaryMetaZnode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server")); // check that the data in the znode is parseable (this would also mean the znode exists) - byte[] data = ZKUtil.getData(zkw, primaryMetaZnode); + byte[] data = ZooKeeperUtil.getData(zkw, primaryMetaZnode); ProtobufUtil.toServerName(data); for (int i = 1; i < 3; i++) { - String secZnode = ZKUtil.joinZNode(baseZNode, + String secZnode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i); String str = zkw.znodePaths.getZNodeForReplica(i); assertTrue(str.equals(secZnode)); // check that the data in the znode is parseable (this would also mean the znode exists) - data = ZKUtil.getData(zkw, secZnode); + data = ZooKeeperUtil.getData(zkw, secZnode); ProtobufUtil.toServerName(data); } } @@ -171,9 +171,9 @@ public class TestMetaWithReplicas { String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String primaryMetaZnode = ZKUtil.joinZNode(baseZNode, + String primaryMetaZnode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server")); - byte[] data = ZKUtil.getData(zkw, primaryMetaZnode); + byte[] data = ZooKeeperUtil.getData(zkw, primaryMetaZnode); ServerName primary = ProtobufUtil.toServerName(data); TableName TABLE = TableName.valueOf("testShutdownHandling"); @@ -368,7 +368,7 @@ public class TestMetaWithReplicas { HBaseFsckRepair.closeRegionSilentlyAndWait(c, rl.getRegionLocation(2).getServerName(), rl.getRegionLocation(2).getRegionInfo()); ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); - ZKUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(2)); + ZooKeeperUtil.deleteNode(zkw, zkw.znodePaths.getZNodeForReplica(2)); // check that problem exists HBaseFsck hbck = doFsck(TEST_UTIL.getConfiguration(), false); assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.UNKNOWN,ERROR_CODE.NO_META_REGION}); @@ -402,10 +402,10 @@ public class TestMetaWithReplicas { ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String primaryMetaZnode = ZKUtil.joinZNode(baseZNode, + String primaryMetaZnode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server")); // check that the data in the znode is parseable (this would also mean the znode exists) - byte[] data = ZKUtil.getData(zkw, primaryMetaZnode); + byte[] data = ZooKeeperUtil.getData(zkw, primaryMetaZnode); ServerName currentServer = ProtobufUtil.toServerName(data); Collection liveServers = TEST_UTIL.getAdmin() .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers(); @@ -427,7 +427,7 @@ public class TestMetaWithReplicas { final int max = 10000; do { Thread.sleep(10); - data = ZKUtil.getData(zkw, primaryMetaZnode); + data = ZooKeeperUtil.getData(zkw, primaryMetaZnode); currentServer = ProtobufUtil.toServerName(data); i++; } while (!moveToServer.equals(currentServer) && i < max); //wait for 10 seconds overall diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java index 7041c92da8..4f967f33df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java index e779706ebb..919fbfa164 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 119c22547f..539e512041 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -25,7 +25,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -38,7 +37,7 @@ import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -183,7 +182,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { /** * @param sn Name of this mock regionserver * @throws IOException - * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException + * @throws ZooKeeperConnectionException */ MockRegionServer(final Configuration conf, final ServerName sn) throws ZooKeeperConnectionException, IOException { 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 f1feef55d5..9633e7bf02 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 @@ -41,8 +41,8 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; @@ -73,8 +73,8 @@ public class TestActiveMasterManager { ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "testActiveMasterManagerFromZK", null, true); try { - ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); - ZKUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) {} // Create the master node with a dummy address @@ -115,8 +115,8 @@ public class TestActiveMasterManager { ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "testActiveMasterManagerFromZK", null, true); try { - ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); - ZKUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) {} // Create the master node with a dummy address @@ -135,7 +135,7 @@ public class TestActiveMasterManager { ClusterStatusTracker clusterStatusTracker = ms1.getClusterStatusTracker(); clusterStatusTracker.setClusterUp(); - activeMasterManager.blockUntilBecomingActiveMaster(100, + activeMasterManager.blockUntilBecomingActiveMaster(100, Mockito.mock(MonitoredTask.class)); assertTrue(activeMasterManager.clusterHasActiveMaster.get()); assertMaster(zk, firstMasterAddress); @@ -165,7 +165,7 @@ public class TestActiveMasterManager { zk.registerListener(listener); LOG.info("Deleting master node"); - ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); // Wait for the node to be deleted LOG.info("Waiting for active master manager to be notified"); @@ -185,7 +185,7 @@ public class TestActiveMasterManager { assertTrue(t.isActiveMaster); LOG.info("Deleting master node"); - ZKUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); + ZooKeeperUtil.deleteNode(zk, zk.znodePaths.masterAddressZNode); } /** @@ -193,7 +193,7 @@ public class TestActiveMasterManager { * @param zk * @param thisMasterAddress * @throws KeeperException - * @throws IOException + * @throws IOException */ private void assertMaster(ZooKeeperWatcher zk, ServerName expectedAddress) @@ -269,7 +269,7 @@ public class TestActiveMasterManager { @Override public void abort(final String msg, final Throwable t) {} - + @Override public boolean isAborted() { return false; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java index 4c4a8edf84..6f03cd953c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java @@ -47,9 +47,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; -import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker; -import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.Assert; @@ -322,7 +320,7 @@ public class TestAssignmentListener { "zkWatcher-NewServerDrainTest", abortable, true); String baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - String drainingZNode = ZKUtil.joinZNode(baseZNode, + String drainingZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.draining.rs", "draining")); HMaster master = Mockito.mock(HMaster.class); @@ -348,8 +346,8 @@ public class TestAssignmentListener { // Create draining znodes for the draining servers, which would have been // performed when the previous HMaster was running. for (ServerName sn : drainingServers) { - String znode = ZKUtil.joinZNode(drainingZNode, sn.getServerName()); - ZKUtil.createAndFailSilent(zooKeeper, znode); + String znode = ZooKeeperUtil.joinZNode(drainingZNode, sn.getServerName()); + ZooKeeperUtil.createAndFailSilent(zooKeeper, znode); } // Now, we follow the same order of steps that the HMaster does to setup 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 eafc41295f..73fdccddca 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 @@ -105,7 +105,7 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.After; @@ -207,7 +207,7 @@ public class TestDistributedLogSplitting { TEST_UTIL.shutdownMiniHBaseCluster(); } finally { TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true); - ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase"); + ZooKeeperUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase"); } } @@ -714,8 +714,8 @@ public class TestDistributedLogSplitting { slm.markRegionsRecovering(firstFailedServer, regionSet); slm.markRegionsRecovering(secondFailedServer, regionSet); - List recoveringRegions = ZKUtil.listChildrenNoWatch(zkw, - ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, region.getEncodedName())); + List recoveringRegions = ZooKeeperUtil.listChildrenNoWatch(zkw, + ZooKeeperUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, region.getEncodedName())); assertEquals(recoveringRegions.size(), 2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java index 64d5a0292e..66070e822f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -67,7 +67,7 @@ public class TestHMasterRPCException { testUtil.startMiniZKCluster(); ZooKeeperWatcher watcher = testUtil.getZooKeeperWatcher(); - ZKUtil.createWithParents(watcher, watcher.znodePaths.masterAddressZNode, Bytes.toBytes("fake:123")); + ZooKeeperUtil.createWithParents(watcher, watcher.znodePaths.masterAddressZNode, Bytes.toBytes("fake:123")); master = new HMaster(conf); rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT); } 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 25671fcce0..d67f717ded 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 @@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.MetaMockingUtil; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; @@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.zookeeper.KeeperException; @@ -117,7 +117,7 @@ public class TestMasterNoCluster { return false; } }); - ZKUtil.deleteNodeRecursively(zkw, zkw.znodePaths.baseZNode); + ZooKeeperUtil.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 8641b20e4b..953c586b23 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 @@ -26,17 +26,14 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SplitLogTask; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs.Ids; @@ -83,23 +80,24 @@ public class TestMasterWalManager { zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, walPath), new SplitLogTask.Owned(inRecoveryServerName).toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - String staleRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, staleRegion); - ZKUtil.createWithParents(zkw, staleRegionPath); - String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, + String staleRegionPath = ZooKeeperUtil + .joinZNode(zkw.znodePaths.recoveringRegionsZNode, staleRegion); + ZooKeeperUtil.createWithParents(zkw, staleRegionPath); + String inRecoveringRegionPath = ZooKeeperUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, failedRegion); - inRecoveringRegionPath = ZKUtil.joinZNode(inRecoveringRegionPath, + inRecoveringRegionPath = ZooKeeperUtil.joinZNode(inRecoveringRegionPath, inRecoveryServerName.getServerName()); - ZKUtil.createWithParents(zkw, inRecoveringRegionPath); + ZooKeeperUtil.createWithParents(zkw, inRecoveringRegionPath); Set servers = new HashSet<>(); servers.add(previouselyFaildServerName); mwm.removeStaleRecoveringRegionsFromZK(servers); // verification - assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1); - assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1); + assertFalse(ZooKeeperUtil.checkExists(zkw, staleRegionPath) != -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, inRecoveringRegionPath) != -1); - ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.recoveringRegionsZNode); - ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.splitLogZNode); + ZooKeeperUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.recoveringRegionsZNode); + ZooKeeperUtil.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 d50e4a3876..20a7000b0b 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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -95,8 +95,8 @@ 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().znodePaths.rsZNode, metaServerName.toString()); - ZKUtil.deleteNode(master.getZooKeeper(), rsEphemeralNodePath); + ZooKeeperUtil.joinZNode(master.getZooKeeper().znodePaths.rsZNode, metaServerName.toString()); + ZooKeeperUtil.deleteNode(master.getZooKeeper(), rsEphemeralNodePath); // Wait for SSH to finish final ServerManager serverManager = master.getServerManager(); final ServerName priorMetaServerName = metaServerName; 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 62157907e0..51cb0387cd 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 @@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -133,12 +134,12 @@ public class TestSplitLogManager { new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null); master = new DummyMasterServices(zkw, conf); - ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); - ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); - assertTrue(ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) != -1); + ZooKeeperUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); + ZooKeeperUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); + assertTrue(ZooKeeperUtil.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); + ZooKeeperUtil.createAndFailSilent(zkw, zkw.znodePaths.splitLogZNode); + assertTrue(ZooKeeperUtil.checkExists(zkw, zkw.znodePaths.splitLogZNode) != -1); LOG.debug(zkw.znodePaths.splitLogZNode + " created"); resetCounters(); @@ -208,7 +209,7 @@ public class TestSplitLogManager { String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name); NodeCreationListener listener = new NodeCreationListener(zkw, tasknode); zkw.registerListener(listener); - ZKUtil.watchAndCheckExists(zkw, tasknode); + ZooKeeperUtil.watchAndCheckExists(zkw, tasknode); slm.enqueueSplitTask(name, batch); assertEquals(1, batch.installed); @@ -234,7 +235,7 @@ public class TestSplitLogManager { String tasknode = submitTaskAndWait(batch, "foo/1"); - byte[] data = ZKUtil.getData(zkw, tasknode); + byte[] data = ZooKeeperUtil.getData(zkw, tasknode); SplitLogTask slt = SplitLogTask.parseFrom(data); LOG.info("Task node created " + slt.toString()); assertTrue(slt.isUnassigned(master.getServerName())); @@ -273,7 +274,7 @@ public class TestSplitLogManager { SplitLogTask slt = new SplitLogTask.Unassigned(master.getServerName(), this.mode); zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - int version = ZKUtil.checkExists(zkw, tasknode); + int version = ZooKeeperUtil.checkExists(zkw, tasknode); slm = new SplitLogManager(master, conf); waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2); @@ -290,7 +291,7 @@ public class TestSplitLogManager { assertEquals(0, task.unforcedResubmits.get()); assertTrue(task.isOrphan()); assertTrue(task.isUnassigned()); - assertTrue(ZKUtil.checkExists(zkw, tasknode) > version); + assertTrue(ZooKeeperUtil.checkExists(zkw, tasknode) > version); } @Test (timeout=180000) @@ -301,24 +302,24 @@ public class TestSplitLogManager { TaskBatch batch = new TaskBatch(); String tasknode = submitTaskAndWait(batch, "foo/1"); - int version = ZKUtil.checkExists(zkw, tasknode); + int version = ZooKeeperUtil.checkExists(zkw, tasknode); final ServerName worker1 = ServerName.valueOf("worker1,1,1"); final ServerName worker2 = ServerName.valueOf("worker2,1,1"); final ServerName worker3 = ServerName.valueOf("worker3,1,1"); SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); waitForCounter(tot_mgr_heartbeat, 0, 1, to/2); waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2); - int version1 = ZKUtil.checkExists(zkw, tasknode); + int version1 = ZooKeeperUtil.checkExists(zkw, tasknode); assertTrue(version1 > version); slt = new SplitLogTask.Owned(worker2, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); waitForCounter(tot_mgr_heartbeat, 1, 2, to/2); waitForCounter(tot_mgr_resubmit, 1, 2, to + to/2); - int version2 = ZKUtil.checkExists(zkw, tasknode); + int version2 = ZooKeeperUtil.checkExists(zkw, tasknode); assertTrue(version2 > version1); slt = new SplitLogTask.Owned(worker3, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); waitForCounter(tot_mgr_heartbeat, 2, 3, to/2); waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to/2); Thread.sleep(to + to/2); @@ -333,10 +334,10 @@ public class TestSplitLogManager { TaskBatch batch = new TaskBatch(); String tasknode = submitTaskAndWait(batch, "foo/1"); - int version = ZKUtil.checkExists(zkw, tasknode); + int version = ZooKeeperUtil.checkExists(zkw, tasknode); final ServerName worker1 = ServerName.valueOf("worker1,1,1"); SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); waitForCounter(tot_mgr_heartbeat, 0, 1, to/2); waitForCounter(new Expr() { @Override @@ -345,9 +346,9 @@ public class TestSplitLogManager { } }, 0, 1, 5*60000); // wait long enough Assert.assertEquals("Could not run test. Lost ZK connection?", 0, tot_mgr_resubmit_failed.sum()); - int version1 = ZKUtil.checkExists(zkw, tasknode); + int version1 = ZooKeeperUtil.checkExists(zkw, tasknode); assertTrue(version1 > version); - byte[] taskstate = ZKUtil.getData(zkw, tasknode); + byte[] taskstate = ZooKeeperUtil.getData(zkw, tasknode); slt = SplitLogTask.parseFrom(taskstate); assertTrue(slt.isUnassigned(master.getServerName())); @@ -363,14 +364,14 @@ public class TestSplitLogManager { String tasknode = submitTaskAndWait(batch, "foo/1"); final ServerName worker1 = ServerName.valueOf("worker1,1,1"); SplitLogTask slt = new SplitLogTask.Done(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); synchronized (batch) { while (batch.installed != batch.done) { batch.wait(); } } waitForCounter(tot_mgr_task_deleted, 0, 1, to/2); - assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, tasknode) == -1); } @Test (timeout=180000) @@ -384,7 +385,7 @@ public class TestSplitLogManager { String tasknode = submitTaskAndWait(batch, "foo/1"); final ServerName worker1 = ServerName.valueOf("worker1,1,1"); SplitLogTask slt = new SplitLogTask.Err(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); synchronized (batch) { while (batch.installed != batch.error) { @@ -392,7 +393,7 @@ public class TestSplitLogManager { } } waitForCounter(tot_mgr_task_deleted, 0, 1, to/2); - assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, tasknode) == -1); conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLogManagerCoordination.DEFAULT_MAX_RESUBMIT); } @@ -409,15 +410,15 @@ public class TestSplitLogManager { assertEquals(tot_mgr_resubmit.sum(), 0); SplitLogTask slt = new SplitLogTask.Resigned(worker1, this.mode); assertEquals(tot_mgr_resubmit.sum(), 0); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); - ZKUtil.checkExists(zkw, tasknode); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.checkExists(zkw, tasknode); // Could be small race here. if (tot_mgr_resubmit.sum() == 0) { waitForCounter(tot_mgr_resubmit, 0, 1, to/2); } assertEquals(tot_mgr_resubmit.sum(), 1); - byte[] taskstate = ZKUtil.getData(zkw, tasknode); + byte[] taskstate = ZooKeeperUtil.getData(zkw, tasknode); slt = SplitLogTask.parseFrom(taskstate); assertTrue(slt.isUnassigned(master.getServerName())); } @@ -446,7 +447,7 @@ public class TestSplitLogManager { Thread.sleep(100); final ServerName worker2 = ServerName.valueOf("worker1,1,1"); slt = new SplitLogTask.Owned(worker2, this.mode); - ZKUtil.setData(zkw, tasknode1, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode1, slt.toByteArray()); } // since we have stopped heartbeating the owned node therefore it should @@ -467,10 +468,10 @@ public class TestSplitLogManager { TaskBatch batch = new TaskBatch(); String tasknode = submitTaskAndWait(batch, "foo/1"); - int version = ZKUtil.checkExists(zkw, tasknode); + int version = ZooKeeperUtil.checkExists(zkw, tasknode); final ServerName worker1 = ServerName.valueOf("worker1,1,1"); SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); if (tot_mgr_heartbeat.sum() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2); slm.handleDeadWorker(worker1); if (tot_mgr_resubmit.sum() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to+to/2); @@ -478,9 +479,9 @@ public class TestSplitLogManager { waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, to + to/2); } - int version1 = ZKUtil.checkExists(zkw, tasknode); + int version1 = ZooKeeperUtil.checkExists(zkw, tasknode); assertTrue(version1 > version); - byte[] taskstate = ZKUtil.getData(zkw, tasknode); + byte[] taskstate = ZooKeeperUtil.getData(zkw, tasknode); slt = SplitLogTask.parseFrom(taskstate); assertTrue(slt.isUnassigned(master.getServerName())); return; @@ -495,7 +496,7 @@ public class TestSplitLogManager { final ServerName worker1 = ServerName.valueOf("worker1,1,1"); SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode); - ZKUtil.setData(zkw, tasknode, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, tasknode, slt.toByteArray()); if (tot_mgr_heartbeat.sum() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2); // Not yet resubmitted. @@ -547,7 +548,7 @@ public class TestSplitLogManager { SplitLogTask slt = new SplitLogTask.Done(worker1, RecoveryMode.LOG_SPLITTING); boolean encounteredZKException = false; try { - ZKUtil.setData(zkw, entry.getKey(), slt.toByteArray()); + ZooKeeperUtil.setData(zkw, entry.getKey(), slt.toByteArray()); } catch (KeeperException e) { LOG.warn(e); encounteredZKException = true; @@ -575,9 +576,9 @@ public class TestSplitLogManager { LOG.info("testRecoveryRegionRemovedFromZK"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); String nodePath = - ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, + ZooKeeperUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); - ZKUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L)); + ZooKeeperUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L)); slm = new SplitLogManager(master, conf); slm.removeStaleRecoveringRegions(null); 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 37def1ba40..ab0c58d750 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 @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -72,15 +72,15 @@ public class TestTableStateManager { private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName, final ZooKeeperProtos.DeprecatedTableState.State state) throws KeeperException, IOException { - String znode = ZKUtil.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString()); - if (ZKUtil.checkExists(watcher, znode) == -1) { - ZKUtil.createAndFailSilent(watcher, znode); + String znode = ZooKeeperUtil.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString()); + if (ZooKeeperUtil.checkExists(watcher, znode) == -1) { + ZooKeeperUtil.createAndFailSilent(watcher, znode); } ZooKeeperProtos.DeprecatedTableState.Builder builder = ZooKeeperProtos.DeprecatedTableState.newBuilder(); builder.setState(state); byte[] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray()); - ZKUtil.setData(watcher, znode, data); + ZooKeeperUtil.setData(watcher, znode, data); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index b5ca894b04..66fe7d61b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationQueues; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java index be7f35ec18..b8f7180647 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java index 8b947ee234..7ccf9f7e78 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -120,15 +120,15 @@ public class TestZKProcedureControllers { // set a prepare node from a 'coordinator' String prepare = ZKProcedureUtil.getAcquireBarrierNode(controller.getZkController(), operationName); - ZKUtil.createSetData(watcher, prepare, ProtobufUtil.prependPBMagic(data)); + ZooKeeperUtil.createSetData(watcher, prepare, ProtobufUtil.prependPBMagic(data)); // wait for the operation to be prepared prepared.await(); // create the commit node so we update the operation to enter the commit phase String commit = ZKProcedureUtil.getReachedBarrierNode(controller.getZkController(), operationName); LOG.debug("Found prepared, posting commit node:" + commit); - ZKUtil.createAndFailSilent(watcher, commit); - LOG.debug("Commit node:" + commit + ", exists:" + ZKUtil.checkExists(watcher, commit)); + ZooKeeperUtil.createAndFailSilent(watcher, commit); + LOG.debug("Commit node:" + commit + ", exists:" + ZooKeeperUtil.checkExists(watcher, commit)); committed.await(); verify(monitor, never()).receive(Mockito.any(ForeignException.class)); @@ -136,9 +136,9 @@ public class TestZKProcedureControllers { // verify(member, never()).getManager().controllerConnectionFailure(Mockito.anyString(), // Mockito.any(IOException.class)); // cleanup after the test - ZKUtil.deleteNodeRecursively(watcher, controller.getZkController().getBaseZnode()); - assertEquals("Didn't delete prepare node", -1, ZKUtil.checkExists(watcher, prepare)); - assertEquals("Didn't delete commit node", -1, ZKUtil.checkExists(watcher, commit)); + ZooKeeperUtil.deleteNodeRecursively(watcher, controller.getZkController().getBaseZnode()); + assertEquals("Didn't delete prepare node", -1, ZooKeeperUtil.checkExists(watcher, prepare)); + assertEquals("Didn't delete commit node", -1, ZooKeeperUtil.checkExists(watcher, commit)); } @Test(timeout = 60000) @@ -350,9 +350,9 @@ public class TestZKProcedureControllers { String prepare = ZKProcedureUtil.getAcquireBarrierNode(controller, operationName); String commit = ZKProcedureUtil.getReachedBarrierNode(controller, operationName); String abort = ZKProcedureUtil.getAbortNode(controller, operationName); - assertEquals("Didn't delete prepare node", -1, ZKUtil.checkExists(watcher, prepare)); - assertEquals("Didn't delete commit node", -1, ZKUtil.checkExists(watcher, commit)); - assertEquals("Didn't delete abort node", -1, ZKUtil.checkExists(watcher, abort)); + assertEquals("Didn't delete prepare node", -1, ZooKeeperUtil.checkExists(watcher, prepare)); + assertEquals("Didn't delete commit node", -1, ZooKeeperUtil.checkExists(watcher, commit)); + assertEquals("Didn't delete abort node", -1, ZooKeeperUtil.checkExists(watcher, abort)); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 5e212f83c3..5eaad16a0f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; import org.apache.hadoop.hbase.wal.FSHLogProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALProvider; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.After; import org.junit.Before; @@ -124,7 +124,7 @@ public class TestCompactionInDeadRegionServer { HRegion region = (HRegion) rsToSuspend.getRegions(TABLE_NAME).get(0); ZooKeeperWatcher watcher = UTIL.getZooKeeperWatcher(); watcher.getRecoverableZooKeeper().delete( - ZKUtil.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()), + ZooKeeperUtil.joinZNode(watcher.getZNodePaths().rsZNode, rsToSuspend.getServerName().toString()), -1); UTIL.waitFor(60000, 1000, new ExplainingPredicate() { 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 84865cc84c..3fe96ca561 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 @@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -82,7 +82,7 @@ public class TestMasterAddressTracker { throws Exception { ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null); - ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode); + ZooKeeperUtil.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 872fec6722..615334760f 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 @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.After; import org.junit.Before; @@ -100,7 +100,7 @@ public class TestRegionServerHostname { TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); try { ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); - List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); + List servers = ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); // there would be NUM_RS+1 children - one for the master assertTrue(servers.size() == NUM_RS + (LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration())? 1: 0)); @@ -161,7 +161,7 @@ public class TestRegionServerHostname { boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration()); int expectedRS = NUM_RS + (tablesOnMaster? 1: 0); try (ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher()) { - List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); + List servers = ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); assertEquals(expectedRS, servers.size()); } } 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 40077f9013..f93f7c4cf9 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 @@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -198,16 +198,16 @@ public class TestSplitLogWorker { zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "split-log-worker-tests", null); ds = new DummyServer(zkw, conf); - ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); - ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); - assertThat(ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode), not (is(-1))); + ZooKeeperUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); + ZooKeeperUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); + assertThat(ZooKeeperUtil.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))); + ZooKeeperUtil.createAndFailSilent(zkw, zkw.znodePaths.splitLogZNode); + assertThat(ZooKeeperUtil.checkExists(zkw, zkw.znodePaths.splitLogZNode), 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))); + ZooKeeperUtil.createAndFailSilent(zkw, zkw.znodePaths.rsZNode); + assertThat(ZooKeeperUtil.checkExists(zkw, zkw.znodePaths.rsZNode), not (is(-1))); SplitLogCounters.resetCounters(); executorService = new ExecutorService("TestSplitLogWorker"); @@ -260,7 +260,7 @@ public class TestSplitLogWorker { slw.start(); try { waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME); - byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS)); + byte [] bytes = ZooKeeperUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS)); SplitLogTask slt = SplitLogTask.parseFrom(bytes); assertTrue(slt.isOwned(RS)); } finally { @@ -304,7 +304,7 @@ public class TestSplitLogWorker { assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned, 0, 1, WAIT_TIME, false) || SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.sum() == 1); - byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT)); + byte [] bytes = ZooKeeperUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT)); SplitLogTask slt = SplitLogTask.parseFrom(bytes); assertTrue(slt.isOwned(SVR1) || slt.isOwned(SVR2)); } finally { @@ -335,11 +335,11 @@ public class TestSplitLogWorker { waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME); assertEquals(1, slw.getTaskReadySeq()); - byte [] bytes = ZKUtil.getData(zkw, PATH); + byte [] bytes = ZooKeeperUtil.getData(zkw, PATH); SplitLogTask slt = SplitLogTask.parseFrom(bytes); assertTrue(slt.isOwned(SRV)); slt = new SplitLogTask.Owned(MANAGER, this.mode); - ZKUtil.setData(zkw, PATH, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, PATH, slt.toByteArray()); waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME); } finally { stopSplitLogWorker(slw); @@ -377,12 +377,12 @@ public class TestSplitLogWorker { // preempt the first task, have it owned by another worker final ServerName anotherWorker = ServerName.valueOf("another-worker,1,1"); SplitLogTask slt = new SplitLogTask.Owned(anotherWorker, this.mode); - ZKUtil.setData(zkw, PATH1, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, PATH1, slt.toByteArray()); waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME); waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, WAIT_TIME); assertEquals(2, slw.getTaskReadySeq()); - byte [] bytes = ZKUtil.getData(zkw, PATH2); + byte [] bytes = ZooKeeperUtil.getData(zkw, PATH2); slt = SplitLogTask.parseFrom(bytes); assertTrue(slt.isOwned(SRV)); } finally { @@ -410,7 +410,7 @@ public class TestSplitLogWorker { // now the worker is busy doing the above task // preempt the task, have it owned by another worker - ZKUtil.setData(zkw, task, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, task, slt.toByteArray()); waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME); // create a RESCAN node @@ -422,11 +422,11 @@ public class TestSplitLogWorker { // RESCAN node might not have been processed if the worker became busy // with the above task. preempt the task again so that now the RESCAN // node is processed - ZKUtil.setData(zkw, task, slt.toByteArray()); + ZooKeeperUtil.setData(zkw, task, slt.toByteArray()); 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.znodePaths.splitLogZNode); + List nodes = ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.splitLogZNode); LOG.debug(nodes); int num = 0; for (String node : nodes) { @@ -434,7 +434,8 @@ 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.znodePaths.splitLogZNode, fn)); + byte [] data = ZooKeeperUtil.getData(zkw, ZooKeeperUtil + .joinZNode(zkw.znodePaths.splitLogZNode, fn)); slt = SplitLogTask.parseFrom(data); assertTrue(slt.toString(), slt.isDone(SRV)); } @@ -463,7 +464,7 @@ public class TestSplitLogWorker { try { waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, maxTasks, WAIT_TIME); for (int i = 0; i < maxTasks; i++) { - byte[] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i)); + byte[] bytes = ZooKeeperUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i)); SplitLogTask slt = SplitLogTask.parseFrom(bytes); assertTrue(slt.isOwned(RS)); } @@ -490,9 +491,9 @@ public class TestSplitLogWorker { RegionServerServices mockedRS = getRegionServer(RS); // create two RS nodes - String rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS.getServerName()); + String rsPath = ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, RS.getServerName()); zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); - rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS2.getServerName()); + rsPath = ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, RS2.getServerName()); zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); for (int i = 0; i < maxTasks; i++) { @@ -507,7 +508,7 @@ public class TestSplitLogWorker { int acquiredTasks = 0; waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 2, WAIT_TIME); for (int i = 0; i < maxTasks; i++) { - byte[] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i)); + byte[] bytes = ZooKeeperUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i)); SplitLogTask slt = SplitLogTask.parseFrom(bytes); if (slt.isOwned(RS)) { acquiredTasks++; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 8533004353..d3bd358381 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Delete; @@ -829,7 +829,7 @@ public class TestSplitTransactionOnCluster { * @return Index of the server hosting the single table region * @throws UnknownRegionException * @throws MasterNotRunningException - * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException + * @throws ZooKeeperConnectionException * @throws InterruptedException */ private int ensureTableRegionNotOnSameServerAsMeta(final Admin admin, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 457a5d062d..d2b54cb752 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -74,7 +74,7 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileTestUtil; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.After; import org.junit.Before; @@ -204,9 +204,10 @@ public class TestMasterReplication { rollWALAndWait(utilities[0], table.getTableName(), row); ZooKeeperWatcher zkw = utilities[0].getZooKeeperWatcher(); String queuesZnode = - ZKUtil.joinZNode(zkw.getZNodePaths().baseZNode, ZKUtil.joinZNode("replication", "rs")); + ZooKeeperUtil + .joinZNode(zkw.getZNodePaths().baseZNode, ZooKeeperUtil.joinZNode("replication", "rs")); List listChildrenNoWatch = - ZKUtil.listChildrenNoWatch(zkw, ZKUtil.joinZNode(queuesZnode, rsName.toString())); + ZooKeeperUtil.listChildrenNoWatch(zkw, ZooKeeperUtil.joinZNode(queuesZnode, rsName.toString())); assertEquals(0, listChildrenNoWatch.size()); } 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 2fe09afacf..826a733b6c 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 @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -88,7 +88,7 @@ public class TestReplicationStateHBaseImpl { utility.startMiniCluster(); zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - replicationZNode = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); + replicationZNode = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); } @Before @@ -387,7 +387,7 @@ public class TestReplicationStateHBaseImpl { @After public void tearDown() throws KeeperException, IOException { - ZKUtil.deleteNodeRecursively(zkw, replicationZNode); + ZooKeeperUtil.deleteNodeRecursively(zkw, replicationZNode); } @AfterClass 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 7d586ad184..03f1565ac9 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 @@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.After; @@ -69,7 +69,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.znodePaths.baseZNode, replicationZNodeName); + replicationZNode = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); KEY_ONE = initPeerClusterState("/hbase1"); KEY_TWO = initPeerClusterState("/hbase2"); } @@ -80,8 +80,8 @@ 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.znodePaths.rsZNode, "hostname1.example.org:1234"); - ZKUtil.createWithParents(zkw1, fakeRs); + String fakeRs = ZooKeeperUtil.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234"); + ZooKeeperUtil.createWithParents(zkw1, fakeRs); ZKClusterId.setClusterId(zkw1, new ClusterId()); return ZKConfig.getZooKeeperClusterKey(testConf); } @@ -111,7 +111,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic { @After public void tearDown() throws KeeperException, IOException { - ZKUtil.deleteNodeRecursively(zkw, replicationZNode); + ZooKeeperUtil.deleteNodeRecursively(zkw, replicationZNode); } @AfterClass @@ -126,13 +126,14 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic { @Test public void testIsPeerPath_PathToChildOfPeerNode() { - String peerChild = ZKUtil.joinZNode(ZKUtil.joinZNode(rqZK.peersZNode, "1"), "child"); + String peerChild = ZooKeeperUtil + .joinZNode(ZooKeeperUtil.joinZNode(rqZK.peersZNode, "1"), "child"); assertFalse(rqZK.isPeerPath(peerChild)); } @Test public void testIsPeerPath_ActualPeerPath() { - String peerPath = ZKUtil.joinZNode(rqZK.peersZNode, "1"); + String peerPath = ZooKeeperUtil.joinZNode(rqZK.peersZNode, "1"); assertTrue(rqZK.isPeerPath(peerPath)); } 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 1e6e153eeb..d991162e84 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 @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.Before; @@ -82,13 +82,13 @@ public class TestReplicationTrackerZKImpl { utility.startMiniZKCluster(); conf = utility.getConfiguration(); ZooKeeperWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility); - ZKUtil.createWithParents(zk, zk.znodePaths.rsZNode); + ZooKeeperUtil.createWithParents(zk, zk.znodePaths.rsZNode); } @Before public void setUp() throws Exception { zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); - String fakeRs1 = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"); + String fakeRs1 = ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"); try { ZKClusterId.setClusterId(zkw, new ClusterId()); rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw); @@ -116,32 +116,38 @@ public class TestReplicationTrackerZKImpl { assertEquals(0, rt.getListOfRegionServers().size()); // 1 region server - ZKUtil.createWithParents(zkw, - ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234")); + ZooKeeperUtil.createWithParents(zkw, + ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234")); assertEquals(1, rt.getListOfRegionServers().size()); // 2 region servers - ZKUtil.createWithParents(zkw, - ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); + ZooKeeperUtil.createWithParents(zkw, + ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); assertEquals(2, rt.getListOfRegionServers().size()); // 1 region server - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); + ZooKeeperUtil + .deleteNode(zkw, ZooKeeperUtil + .joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); assertEquals(1, rt.getListOfRegionServers().size()); // 0 region server - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234")); + ZooKeeperUtil + .deleteNode(zkw, ZooKeeperUtil + .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.znodePaths.rsZNode, "hostname2.example.org:1234"), + ZooKeeperUtil.createAndWatch(zkw, + ZooKeeperUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"), HConstants.EMPTY_BYTE_ARRAY); rt.registerListener(new DummyReplicationListener()); // delete one - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); + ZooKeeperUtil + .deleteNode(zkw, ZooKeeperUtil + .joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); // wait for event while (rsRemovedCount.get() < 1) { Thread.sleep(5); @@ -170,7 +176,7 @@ public class TestReplicationTrackerZKImpl { rp.disablePeer("5"); int tmp = plChangedCount.get(); LOG.info("Peer count=" + tmp); - ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state"); + ZooKeeperUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state"); // wait for event while (plChangedCount.get() <= tmp) { Thread.sleep(100); @@ -189,7 +195,7 @@ public class TestReplicationTrackerZKImpl { int exists = 0; int hyphen = 0; rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); - + try{ rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); }catch(IllegalArgumentException e){ @@ -203,11 +209,11 @@ public class TestReplicationTrackerZKImpl { } assertEquals(1, exists); assertEquals(1, hyphen); - + // clean up rp.unregisterPeer("6"); } - + private class DummyReplicationListener implements ReplicationListener { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java index 8c604f46a8..c635b5dae6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; @@ -99,15 +99,16 @@ public class TestTableCFsUpdater extends TableCFsUpdater { ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); String peerNode = getPeerNode(peerId); - ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); + ZooKeeperUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); String tableCFs = tableName1 + ":cf1,cf2;" + tableName2 + ":cf3;" + tableName3; String tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZooKeeperUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); - ReplicationPeerConfig actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); - String actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); + ReplicationPeerConfig actualRpc = ReplicationSerDeHelper.parsePeerFrom( + ZooKeeperUtil.getData(zkw, peerNode)); + String actualTableCfs = Bytes.toString(ZooKeeperUtil.getData(zkw, tableCFsNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); assertNull(actualRpc.getTableCFsMap()); @@ -117,15 +118,15 @@ public class TestTableCFsUpdater extends TableCFsUpdater { rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); peerNode = getPeerNode(peerId); - ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); + ZooKeeperUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); tableCFs = tableName1 + ":cf1,cf3;" + tableName2 + ":cf2"; tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZooKeeperUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); - actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); + actualTableCfs = Bytes.toString(ZooKeeperUtil.getData(zkw, tableCFsNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); assertNull(actualRpc.getTableCFsMap()); @@ -135,15 +136,15 @@ public class TestTableCFsUpdater extends TableCFsUpdater { rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); peerNode = getPeerNode(peerId); - ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); + ZooKeeperUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); tableCFs = ""; tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZooKeeperUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); - actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); + actualTableCfs = Bytes.toString(ZooKeeperUtil.getData(zkw, tableCFsNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); assertNull(actualRpc.getTableCFsMap()); @@ -153,11 +154,11 @@ public class TestTableCFsUpdater extends TableCFsUpdater { rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); peerNode = getPeerNode(peerId); - ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); + ZooKeeperUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc)); tableCFsNode = getTableCFsNode(peerId); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); - actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); + actualTableCfs = Bytes.toString(ZooKeeperUtil.getData(zkw, tableCFsNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); assertNull(actualRpc.getTableCFsMap()); @@ -167,7 +168,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater { peerId = "1"; peerNode = getPeerNode(peerId); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); Map> tableNameListMap = actualRpc.getTableCFsMap(); assertEquals(3, tableNameListMap.size()); @@ -184,7 +185,7 @@ public class TestTableCFsUpdater extends TableCFsUpdater { peerId = "2"; peerNode = getPeerNode(peerId); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); assertEquals(2, tableNameListMap.size()); @@ -198,14 +199,14 @@ public class TestTableCFsUpdater extends TableCFsUpdater { peerId = "3"; peerNode = getPeerNode(peerId); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); assertNull(tableNameListMap); peerId = "4"; peerNode = getPeerNode(peerId); - actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); + actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZooKeeperUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); assertNull(tableNameListMap); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 8451d694f3..c18f13e602 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -82,7 +82,7 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.After; import org.junit.AfterClass; @@ -150,16 +150,16 @@ public abstract class TestReplicationSourceManager { // The implementing class should set up the conf assertNotNull(conf); zkw = new ZooKeeperWatcher(conf, "test", null); - ZKUtil.createWithParents(zkw, "/hbase/replication"); - ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1"); - ZKUtil.setData(zkw, "/hbase/replication/peers/1", + ZooKeeperUtil.createWithParents(zkw, "/hbase/replication"); + ZooKeeperUtil.createWithParents(zkw, "/hbase/replication/peers/1"); + ZooKeeperUtil.setData(zkw, "/hbase/replication/peers/1", Bytes.toBytes(conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":/1")); - ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state"); - ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state", + ZooKeeperUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state"); + ZooKeeperUtil.setData(zkw, "/hbase/replication/peers/1/peer-state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES); - ZKUtil.createWithParents(zkw, "/hbase/replication/state"); - ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES); + ZooKeeperUtil.createWithParents(zkw, "/hbase/replication/state"); + ZooKeeperUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES); ZKClusterId.setClusterId(zkw, new ClusterId()); FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index ad8cb14067..f45e2d0905 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.TestTableName; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.After; import org.junit.AfterClass; @@ -510,9 +510,11 @@ public class TestAccessController2 extends SecureTestUtil { // Namespace needs this, as they follow the lazy creation of ACL znode. grantOnNamespace(TEST_UTIL, TESTGROUP1_USER1.getShortName(), ns, Action.ADMIN); ZooKeeperWatcher zkw = TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper(); - assertTrue("The acl znode for table should exist", ZKUtil.checkExists(zkw, baseAclZNode + + assertTrue("The acl znode for table should exist", ZooKeeperUtil + .checkExists(zkw, baseAclZNode + table.getNameAsString()) != -1); - assertTrue("The acl znode for namespace should exist", ZKUtil.checkExists(zkw, baseAclZNode + + assertTrue("The acl znode for namespace should exist", ZooKeeperUtil + .checkExists(zkw, baseAclZNode + convertToNamespace(ns)) != -1); revokeFromNamespace(TEST_UTIL, TESTGROUP1_USER1.getShortName(), ns, Action.ADMIN); @@ -520,8 +522,8 @@ public class TestAccessController2 extends SecureTestUtil { deleteNamespace(TEST_UTIL, ns); assertTrue("The acl znode for table should have been deleted", - ZKUtil.checkExists(zkw, baseAclZNode + table.getNameAsString()) == -1); + ZooKeeperUtil.checkExists(zkw, baseAclZNode + table.getNameAsString()) == -1); assertTrue( "The acl znode for namespace should have been deleted", - ZKUtil.checkExists(zkw, baseAclZNode + convertToNamespace(ns)) == -1); + ZooKeeperUtil.checkExists(zkw, baseAclZNode + convertToNamespace(ns)) == -1); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java index b2396819c8..ca9ecc7385 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.Assert; @@ -54,7 +54,7 @@ public class TestZKSecretWatcherRefreshKeys { return abort; } } - + @BeforeClass public static void setupBeforeClass() throws Exception { TEST_UTIL = new HBaseTestingUtility(); @@ -77,17 +77,17 @@ public class TestZKSecretWatcherRefreshKeys { public void testRefreshKeys() throws Exception { Configuration conf = TEST_UTIL.getConfiguration(); ZooKeeperWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable()); - AuthenticationTokenSecretManager keyManager = - new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1", + AuthenticationTokenSecretManager keyManager = + new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1", 60 * 60 * 1000, 60 * 1000); ZKSecretWatcher watcher = new ZKSecretWatcher(conf, zk, keyManager); - ZKUtil.deleteChildrenRecursively(zk, watcher.getKeysParentZNode()); + ZooKeeperUtil.deleteChildrenRecursively(zk, watcher.getKeysParentZNode()); Integer[] keys = { 1, 2, 3, 4, 5, 6 }; for (Integer key : keys) { AuthenticationKey ak = new AuthenticationKey(key, System.currentTimeMillis() + 600 * 1000, null); - ZKUtil.createWithParents(zk, - ZKUtil.joinZNode(watcher.getKeysParentZNode(), key.toString()), + ZooKeeperUtil.createWithParents(zk, + ZooKeeperUtil.joinZNode(watcher.getKeysParentZNode(), key.toString()), Writables.getBytes(ak)); } Assert.assertNull(keyManager.getCurrentKey()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java index db442192a5..237d68eaaf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; public class MockServer implements Server { private static final Log LOG = LogFactory.getLog(MockServer.class); final static ServerName NAME = ServerName.valueOf("MockServer", 123, -1); - + boolean stopped; boolean aborted; final ZooKeeperWatcher zk; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index a5cf0bd4a6..efc7b21884 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.experimental.categories.Category; @@ -162,7 +162,7 @@ public class ProcessBasedLocalHBaseCluster { startMaster(masterPort); } - ZKUtil.waitForBaseZNode(conf); + ZooKeeperUtil.waitForBaseZNode(conf); for (int rsPort : rsPorts) { startRegionServer(rsPort); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java index e71210d067..35c3988668 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java @@ -74,7 +74,7 @@ public class TestRecoverableZooKeeper { ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSetDataVersionMismatchInLoop", abortable, true); String ensemble = ZKConfig.getZKQuorumServersString(conf); - RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw); + RecoverableZooKeeper rzk = ZooKeeperUtil.connect(conf, ensemble, zkw); rzk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); rzk.setData(znode, "OPENING".getBytes(), 0); Field zkField = RecoverableZooKeeper.class.getDeclaredField("zk"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java index 1faf8e52e0..adca75008b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKLeaderManager.java @@ -150,7 +150,7 @@ public class TestZKLeaderManager { assertNotNull("Leader should exist", currentLeader); LOG.debug("Current leader index is "+currentLeader.getIndex()); - byte[] znodeData = ZKUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); + byte[] znodeData = ZooKeeperUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); assertNotNull("Leader znode should contain leader index", znodeData); assertTrue("Leader znode should not be empty", znodeData.length > 0); int storedIndex = Bytes.toInt(znodeData); @@ -168,7 +168,7 @@ public class TestZKLeaderManager { assertNotNull("New leader should exist after abdication", currentLeader); LOG.debug("New leader index is "+currentLeader.getIndex()); - znodeData = ZKUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); + znodeData = ZooKeeperUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); assertNotNull("Leader znode should contain leader index", znodeData); assertTrue("Leader znode should not be empty", znodeData.length > 0); storedIndex = Bytes.toInt(znodeData); @@ -186,7 +186,7 @@ public class TestZKLeaderManager { assertNotNull("New leader should exist after stop", currentLeader); LOG.debug("New leader index is "+currentLeader.getIndex()); - znodeData = ZKUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); + znodeData = ZooKeeperUtil.getData(currentLeader.getWatcher(), LEADER_ZNODE); assertNotNull("Leader znode should contain leader index", znodeData); assertTrue("Leader znode should not be empty", znodeData.length > 0); storedIndex = Bytes.toInt(znodeData); 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 b4ac59c746..8c0c00add2 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 @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.Op; @@ -82,83 +82,83 @@ public class TestZKMulti { @Test (timeout=60000) public void testSimpleMulti() throws Exception { // null multi - ZKUtil.multiOrSequential(zkw, null, false); + ZooKeeperUtil.multiOrSequential(zkw, null, false); // empty multi - ZKUtil.multiOrSequential(zkw, new LinkedList<>(), false); + ZooKeeperUtil.multiOrSequential(zkw, new LinkedList<>(), false); // single create - String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSimpleMulti"); + String path = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testSimpleMulti"); LinkedList singleCreate = new LinkedList<>(); - singleCreate.add(ZKUtilOp.createAndFailSilent(path, new byte[0])); - ZKUtil.multiOrSequential(zkw, singleCreate, false); - assertTrue(ZKUtil.checkExists(zkw, path) != -1); + singleCreate.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path, new byte[0])); + ZooKeeperUtil.multiOrSequential(zkw, singleCreate, false); + assertTrue(ZooKeeperUtil.checkExists(zkw, path) != -1); // single setdata LinkedList singleSetData = new LinkedList<>(); byte [] data = Bytes.toBytes("foobar"); - singleSetData.add(ZKUtilOp.setData(path, data)); - ZKUtil.multiOrSequential(zkw, singleSetData, false); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path), data)); + singleSetData.add(ZooKeeperUtil.ZKUtilOp.setData(path, data)); + ZooKeeperUtil.multiOrSequential(zkw, singleSetData, false); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path), data)); // single delete LinkedList singleDelete = new LinkedList<>(); - singleDelete.add(ZKUtilOp.deleteNodeFailSilent(path)); - ZKUtil.multiOrSequential(zkw, singleDelete, false); - assertTrue(ZKUtil.checkExists(zkw, path) == -1); + singleDelete.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path)); + ZooKeeperUtil.multiOrSequential(zkw, singleDelete, false); + assertTrue(ZooKeeperUtil.checkExists(zkw, path) == -1); } @Test (timeout=60000) public void testComplexMulti() throws Exception { - 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"); + String path1 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti1"); + String path2 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti2"); + String path3 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti3"); + String path4 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti4"); + String path5 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti5"); + String path6 = ZooKeeperUtil.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))); - create4Nodes.add(ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2))); - create4Nodes.add(ZKUtilOp.createAndFailSilent(path3, Bytes.toBytes(path3))); - create4Nodes.add(ZKUtilOp.createAndFailSilent(path4, Bytes.toBytes(path4))); - ZKUtil.multiOrSequential(zkw, create4Nodes, false); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1), Bytes.toBytes(path1))); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path2), Bytes.toBytes(path2))); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path3), Bytes.toBytes(path3))); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path4), Bytes.toBytes(path4))); + create4Nodes.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1))); + create4Nodes.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2))); + create4Nodes.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path3, Bytes.toBytes(path3))); + create4Nodes.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path4, Bytes.toBytes(path4))); + ZooKeeperUtil.multiOrSequential(zkw, create4Nodes, false); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path1), Bytes.toBytes(path1))); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path2), Bytes.toBytes(path2))); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path3), Bytes.toBytes(path3))); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path4), Bytes.toBytes(path4))); // do multiple of each operation (setData, delete, create) LinkedList ops = new LinkedList<>(); // setData - ops.add(ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); - ops.add(ZKUtilOp.setData(path2, Bytes.add(Bytes.toBytes(path2), Bytes.toBytes(path2)))); + ops.add(ZooKeeperUtil.ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); + ops.add(ZooKeeperUtil.ZKUtilOp.setData(path2, Bytes.add(Bytes.toBytes(path2), Bytes.toBytes(path2)))); // delete - ops.add(ZKUtilOp.deleteNodeFailSilent(path3)); - ops.add(ZKUtilOp.deleteNodeFailSilent(path4)); + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path3)); + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path4)); // create - ops.add(ZKUtilOp.createAndFailSilent(path5, Bytes.toBytes(path5))); - ops.add(ZKUtilOp.createAndFailSilent(path6, Bytes.toBytes(path6))); - ZKUtil.multiOrSequential(zkw, ops, false); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1), + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path5, Bytes.toBytes(path5))); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path6, Bytes.toBytes(path6))); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path1), Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path2), + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path2), Bytes.add(Bytes.toBytes(path2), Bytes.toBytes(path2)))); - assertTrue(ZKUtil.checkExists(zkw, path3) == -1); - assertTrue(ZKUtil.checkExists(zkw, path4) == -1); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path5), Bytes.toBytes(path5))); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path6), Bytes.toBytes(path6))); + assertTrue(ZooKeeperUtil.checkExists(zkw, path3) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, path4) == -1); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path5), Bytes.toBytes(path5))); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path6), Bytes.toBytes(path6))); } @Test (timeout=60000) public void testSingleFailure() throws Exception { // try to delete a node that doesn't exist boolean caughtNoNode = false; - String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureZ"); + String path = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureZ"); LinkedList ops = new LinkedList<>(); - ops.add(ZKUtilOp.deleteNodeFailSilent(path)); + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path)); try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NoNodeException nne) { caughtNoNode = true; } @@ -167,9 +167,9 @@ public class TestZKMulti { // try to setData on a node that doesn't exist caughtNoNode = false; ops = new LinkedList<>(); - ops.add(ZKUtilOp.setData(path, Bytes.toBytes(path))); + ops.add(ZooKeeperUtil.ZKUtilOp.setData(path, Bytes.toBytes(path))); try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NoNodeException nne) { caughtNoNode = true; } @@ -178,10 +178,10 @@ public class TestZKMulti { // try to create on a node that already exists boolean caughtNodeExists = false; ops = new LinkedList<>(); - ops.add(ZKUtilOp.createAndFailSilent(path, Bytes.toBytes(path))); - ZKUtil.multiOrSequential(zkw, ops, false); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path, Bytes.toBytes(path))); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NodeExistsException nee) { caughtNodeExists = true; } @@ -191,107 +191,107 @@ 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.znodePaths.baseZNode, "testSingleFailureInMultiA"); - String pathB = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiB"); - String pathC = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiC"); + String pathA = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiA"); + String pathB = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiB"); + String pathC = ZooKeeperUtil.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))); - ops.add(ZKUtilOp.deleteNodeFailSilent(pathC)); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathA, Bytes.toBytes(pathA))); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathB, Bytes.toBytes(pathB))); + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(pathC)); boolean caughtNoNode = false; try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NoNodeException nne) { caughtNoNode = true; } assertTrue(caughtNoNode); // assert that none of the operations succeeded - assertTrue(ZKUtil.checkExists(zkw, pathA) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathB) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathC) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathA) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathB) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathC) == -1); } @Test (timeout=60000) public void testMultiFailure() throws Exception { - String pathX = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureX"); - String pathY = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureY"); - String pathZ = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureZ"); + String pathX = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureX"); + String pathY = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureY"); + String pathZ = ZooKeeperUtil.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); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); // fail one of each create ,setData, delete - String pathV = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureV"); - String pathW = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureW"); + String pathV = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureV"); + String pathW = ZooKeeperUtil.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 - ops.add(ZKUtilOp.deleteNodeFailSilent(pathZ)); // fail -- doesn't exist - ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathV))); // pass - ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathW))); // pass + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- already exists + ops.add(ZooKeeperUtil.ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(pathZ)); // fail -- doesn't exist + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathV))); // pass + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathW))); // pass boolean caughtNodeExists = false; try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NodeExistsException nee) { // check first operation that fails throws exception caughtNodeExists = true; } assertTrue(caughtNodeExists); // check that no modifications were made - assertFalse(ZKUtil.checkExists(zkw, pathX) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathY) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathZ) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathW) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathV) == -1); + assertFalse(ZooKeeperUtil.checkExists(zkw, pathX) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathY) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathZ) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathW) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathV) == -1); // test that with multiple failures, throws an exception corresponding to first failure in list ops = new LinkedList<>(); - ops.add(ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist - ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- exists + ops.add(ZooKeeperUtil.ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- exists boolean caughtNoNode = false; try { - ZKUtil.multiOrSequential(zkw, ops, false); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); } catch (KeeperException.NoNodeException nne) { // check first operation that fails throws exception caughtNoNode = true; } assertTrue(caughtNoNode); // check that no modifications were made - assertFalse(ZKUtil.checkExists(zkw, pathX) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathY) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathZ) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathW) == -1); - assertTrue(ZKUtil.checkExists(zkw, pathV) == -1); + assertFalse(ZooKeeperUtil.checkExists(zkw, pathX) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathY) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathZ) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathW) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, pathV) == -1); } @Test (timeout=60000) public void testRunSequentialOnMultiFailure() throws Exception { - 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"); + String path1 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential1"); + String path2 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential2"); + String path3 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential3"); + String path4 = ZooKeeperUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential4"); // create some nodes that we will use later LinkedList ops = new LinkedList<>(); - ops.add(ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1))); - ops.add(ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2))); - ZKUtil.multiOrSequential(zkw, ops, false); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1))); + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path2, Bytes.toBytes(path2))); + ZooKeeperUtil.multiOrSequential(zkw, ops, false); // test that, even with operations that fail, the ones that would pass will pass // with runSequentialOnMultiFailure ops = new LinkedList<>(); - ops.add(ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); // pass - ops.add(ZKUtilOp.deleteNodeFailSilent(path2)); // pass - ops.add(ZKUtilOp.deleteNodeFailSilent(path3)); // fail -- node doesn't exist - ops.add(ZKUtilOp.createAndFailSilent(path4, + ops.add(ZooKeeperUtil.ZKUtilOp.setData(path1, Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); // pass + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path2)); // pass + ops.add(ZooKeeperUtil.ZKUtilOp.deleteNodeFailSilent(path3)); // fail -- node doesn't exist + ops.add(ZooKeeperUtil.ZKUtilOp.createAndFailSilent(path4, Bytes.add(Bytes.toBytes(path4), Bytes.toBytes(path4)))); // pass - ZKUtil.multiOrSequential(zkw, ops, true); - assertTrue(Bytes.equals(ZKUtil.getData(zkw, path1), + ZooKeeperUtil.multiOrSequential(zkw, ops, true); + assertTrue(Bytes.equals(ZooKeeperUtil.getData(zkw, path1), Bytes.add(Bytes.toBytes(path1), Bytes.toBytes(path1)))); - assertTrue(ZKUtil.checkExists(zkw, path2) == -1); - assertTrue(ZKUtil.checkExists(zkw, path3) == -1); - assertFalse(ZKUtil.checkExists(zkw, path4) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, path2) == -1); + assertTrue(ZooKeeperUtil.checkExists(zkw, path3) == -1); + assertFalse(ZooKeeperUtil.checkExists(zkw, path4) == -1); } /** @@ -303,10 +303,10 @@ public class TestZKMulti { String parentZNode = "/testRootMulti"; createZNodeTree(parentZNode); - ZKUtil.deleteChildrenRecursivelyMultiOrSequential(zkw, true, parentZNode); + ZooKeeperUtil.deleteChildrenRecursivelyMultiOrSequential(zkw, true, parentZNode); assertTrue("Wrongly deleted parent znode!", - ZKUtil.checkExists(zkw, parentZNode) > -1); + ZooKeeperUtil.checkExists(zkw, parentZNode) > -1); List children = zkw.getRecoverableZooKeeper().getChildren( parentZNode, false); assertTrue("Failed to delete child znodes!", 0 == children.size()); @@ -321,8 +321,8 @@ public class TestZKMulti { String parentZNode = "/testdeleteNodeRecursivelyMulti"; createZNodeTree(parentZNode); - ZKUtil.deleteNodeRecursively(zkw, parentZNode); - assertTrue("Parent znode should be deleted.", ZKUtil.checkExists(zkw, parentZNode) == -1); + ZooKeeperUtil.deleteNodeRecursively(zkw, parentZNode); + assertTrue("Parent znode should be deleted.", ZooKeeperUtil.checkExists(zkw, parentZNode) == -1); } @Test(timeout = 60000) @@ -334,11 +334,11 @@ public class TestZKMulti { createZNodeTree(parentZNode2); createZNodeTree(parentZNode3); - ZKUtil.deleteNodeRecursivelyMultiOrSequential(zkw, false, parentZNode1, parentZNode2, + ZooKeeperUtil.deleteNodeRecursivelyMultiOrSequential(zkw, false, parentZNode1, parentZNode2, parentZNode3); - assertTrue("Parent znode 1 should be deleted.", ZKUtil.checkExists(zkw, parentZNode1) == -1); - assertTrue("Parent znode 2 should be deleted.", ZKUtil.checkExists(zkw, parentZNode2) == -1); - assertTrue("Parent znode 3 should be deleted.", ZKUtil.checkExists(zkw, parentZNode3) == -1); + assertTrue("Parent znode 1 should be deleted.", ZooKeeperUtil.checkExists(zkw, parentZNode1) == -1); + assertTrue("Parent znode 2 should be deleted.", ZooKeeperUtil.checkExists(zkw, parentZNode2) == -1); + assertTrue("Parent znode 3 should be deleted.", ZooKeeperUtil.checkExists(zkw, parentZNode3) == -1); } @Test(timeout = 60000) @@ -350,18 +350,18 @@ public class TestZKMulti { createZNodeTree(parentZNode2); createZNodeTree(parentZNode3); - ZKUtil.deleteChildrenRecursivelyMultiOrSequential(zkw, true, parentZNode1, parentZNode2, + ZooKeeperUtil.deleteChildrenRecursivelyMultiOrSequential(zkw, true, parentZNode1, parentZNode2, parentZNode3); - assertTrue("Wrongly deleted parent znode 1!", ZKUtil.checkExists(zkw, parentZNode1) > -1); + assertTrue("Wrongly deleted parent znode 1!", ZooKeeperUtil.checkExists(zkw, parentZNode1) > -1); List children = zkw.getRecoverableZooKeeper().getChildren(parentZNode1, false); assertTrue("Failed to delete child znodes of parent znode 1!", 0 == children.size()); - assertTrue("Wrongly deleted parent znode 2!", ZKUtil.checkExists(zkw, parentZNode2) > -1); + assertTrue("Wrongly deleted parent znode 2!", ZooKeeperUtil.checkExists(zkw, parentZNode2) > -1); children = zkw.getRecoverableZooKeeper().getChildren(parentZNode2, false); assertTrue("Failed to delete child znodes of parent znode 1!", 0 == children.size()); - assertTrue("Wrongly deleted parent znode 3!", ZKUtil.checkExists(zkw, parentZNode3) > -1); + assertTrue("Wrongly deleted parent znode 3!", ZooKeeperUtil.checkExists(zkw, parentZNode3) > -1); children = zkw.getRecoverableZooKeeper().getChildren(parentZNode3, false); assertTrue("Failed to delete child znodes of parent znode 1!", 0 == children.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java index 05ad73e974..beee13fff5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java @@ -51,7 +51,7 @@ public class TestZooKeeperACL { private static ZooKeeperWatcher zkw; private static boolean secureZKAvailable; - + @BeforeClass public static void setUpBeforeClass() throws Exception { File saslConfFile = File.createTempFile("tmp", "jaas.conf"); @@ -76,7 +76,7 @@ public class TestZooKeeperACL { TEST_UTIL.getConfiguration().setInt("hbase.zookeeper.property.maxClientCnxns", 1000); // If Hadoop is missing HADOOP-7070 the cluster will fail to start due to - // the JAAS configuration required by ZK being clobbered by Hadoop + // the JAAS configuration required by ZK being clobbered by Hadoop try { TEST_UTIL.startMiniCluster(); } catch (IOException e) { @@ -112,7 +112,7 @@ public class TestZooKeeperACL { } /** - * Create a node and check its ACL. When authentication is enabled on + * Create a node and check its ACL. When authentication is enabled on * ZooKeeper, all nodes (except /hbase/root-region-server, /hbase/master * and /hbase/hbaseid) should be created so that only the hbase server user * (master or region server user) that created them can access them, and @@ -256,7 +256,7 @@ public class TestZooKeeperACL { return; } - ZKUtil.createWithParents(zkw, "/testACLNode"); + ZooKeeperUtil.createWithParents(zkw, "/testACLNode"); List acls = zkw.getRecoverableZooKeeper().getZooKeeper() .getACL("/testACLNode", new Stat()); assertEquals(acls.size(),1); @@ -270,7 +270,7 @@ public class TestZooKeeperACL { */ @Test public void testIsZooKeeperSecure() throws Exception { - boolean testJaasConfig = ZKUtil.isSecureZooKeeper(new Configuration(TEST_UTIL.getConfiguration())); + boolean testJaasConfig = ZooKeeperUtil.isSecureZooKeeper(new Configuration(TEST_UTIL.getConfiguration())); assertEquals(testJaasConfig, secureZKAvailable); // Define Jaas configuration without ZooKeeper Jaas config File saslConfFile = File.createTempFile("tmp", "fakeJaas.conf"); @@ -281,11 +281,11 @@ public class TestZooKeeperACL { System.setProperty("java.security.auth.login.config", saslConfFile.getAbsolutePath()); - testJaasConfig = ZKUtil.isSecureZooKeeper(new Configuration(TEST_UTIL.getConfiguration())); + testJaasConfig = ZooKeeperUtil.isSecureZooKeeper(new Configuration(TEST_UTIL.getConfiguration())); assertEquals(testJaasConfig, false); saslConfFile.delete(); } - + /** * Check if Programmatic way of setting zookeeper security settings is valid. */ @@ -295,13 +295,13 @@ public class TestZooKeeperACL { javax.security.auth.login.Configuration.setConfiguration(new DummySecurityConfiguration()); Configuration config = new Configuration(HBaseConfiguration.create()); - boolean testJaasConfig = ZKUtil.isSecureZooKeeper(config); + boolean testJaasConfig = ZooKeeperUtil.isSecureZooKeeper(config); assertEquals(testJaasConfig, false); // Now set authentication scheme to Kerberos still it should return false // because no configuration set config.set("hbase.security.authentication", "kerberos"); - testJaasConfig = ZKUtil.isSecureZooKeeper(config); + testJaasConfig = ZooKeeperUtil.isSecureZooKeeper(config); assertEquals(testJaasConfig, false); // Now set programmatic options related to security @@ -309,7 +309,7 @@ public class TestZooKeeperACL { config.set(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL, "dummy"); config.set(HConstants.ZK_SERVER_KEYTAB_FILE, "/dummy/file"); config.set(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL, "dummy"); - testJaasConfig = ZKUtil.isSecureZooKeeper(config); + testJaasConfig = ZooKeeperUtil.isSecureZooKeeper(config); assertEquals(true, testJaasConfig); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java index 368919ceb5..f0d64b7ab5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperMainServer.java @@ -74,8 +74,8 @@ public class TestZooKeeperMainServer { try { ZooKeeperWatcher zkw = htu.getZooKeeperWatcher(); String znode = "/testCommandLineWorks"; - ZKUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY); - ZKUtil.checkExists(zkw, znode); + ZooKeeperUtil.createWithParents(zkw, znode, HConstants.EMPTY_BYTE_ARRAY); + ZooKeeperUtil.checkExists(zkw, znode); boolean exception = false; try { ZooKeeperMainServer.main(new String [] {"-server", @@ -85,7 +85,7 @@ public class TestZooKeeperMainServer { exception = true; } assertTrue(exception); - assertEquals(-1, ZKUtil.checkExists(zkw, znode)); + assertEquals(-1, ZooKeeperUtil.checkExists(zkw, znode)); } finally { htu.shutdownMiniZKCluster(); System.setSecurityManager(null); // or save and restore original 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 87e2309db9..a08864b6ce 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.znodePaths.baseZNode); + ZooKeeperUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode); final String node = - ZKUtil.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString()); + ZooKeeperUtil.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString()); final byte [] dataOne = Bytes.toBytes("dataOne"); final byte [] dataTwo = Bytes.toBytes("dataTwo"); @@ -302,12 +302,12 @@ public class TestZooKeeperNodeTracker { public static class StubAbortable implements Abortable { @Override public void abort(final String msg, final Throwable t) {} - + @Override public boolean isAborted() { return false; } - + } public static class StubWatcher implements Watcher { @@ -322,26 +322,26 @@ public class TestZooKeeperNodeTracker { final ServerName sn = ServerName.valueOf("127.0.0.1:52", 45L); - ZKUtil.createAndFailSilent(zkw, + ZooKeeperUtil.createAndFailSilent(zkw, TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)); final String nodeName = zkw.znodePaths.masterAddressZNode; // Check that we manage the case when there is no data - ZKUtil.createAndFailSilent(zkw, nodeName); + ZooKeeperUtil.createAndFailSilent(zkw, nodeName); MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); - Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null); + Assert.assertFalse(ZooKeeperUtil.getData(zkw, nodeName) == null); // Check that we don't delete if we're not supposed to - ZKUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0)); + ZooKeeperUtil.setData(zkw, nodeName, MasterAddressTracker.toByteArray(sn, 0)); MasterAddressTracker.deleteIfEquals(zkw, ServerName.valueOf("127.0.0.2:52", 45L).toString()); - Assert.assertFalse(ZKUtil.getData(zkw, nodeName) == null); + Assert.assertFalse(ZooKeeperUtil.getData(zkw, nodeName) == null); // Check that we delete when we're supposed to - ZKUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0)); + ZooKeeperUtil.setData(zkw, nodeName,MasterAddressTracker.toByteArray(sn, 0)); MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); - Assert.assertTrue( ZKUtil.getData(zkw, nodeName)== null ); + Assert.assertTrue( ZooKeeperUtil.getData(zkw, nodeName)== null ); // Check that we support the case when the znode does not exist MasterAddressTracker.deleteIfEquals(zkw, sn.toString()); // must not throw an exception diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml new file mode 100644 index 0000000000..00917debcb --- /dev/null +++ b/hbase-zookeeper/pom.xml @@ -0,0 +1,655 @@ + + + + 4.0.0 + + hbase-build-configuration + org.apache.hbase + 3.0.0-SNAPSHOT + ../hbase-build-configuration + + hbase-zookeeper + Apache HBase - Zookeeper + Zookeeper Helpers for HBase + + + + + + + src/test/resources/META-INF/ + META-INF/ + + NOTICE + + true + + + src/test/resources + + **/** + + + + + + org.apache.maven.plugins + maven-site-plugin + + true + + + + + + maven-assembly-plugin + + true + + + + + org.apache.maven.plugins + maven-source-plugin + + + package + + jar + test-jar + + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + + + org.jamon.project.jamonnature + + + org.jamon.project.templateBuilder + org.eclipse.jdt.core.javabuilder + org.jamon.project.markerUpdater + + + + .settings/org.jamon.prefs + # now + eclipse.preferences.version=1 + templateSourceDir=src/main/jamon + templateOutputDir=target/generated-jamon + + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + + + maven-surefire-plugin + + + + listener + org.apache.hadoop.hbase.ResourceCheckerJUnitListener + + + + target/test-classes/webapps + org.apache.hadoop.hbase.shaded. + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + + + + + + org.apache.maven.plugins + maven-antrun-plugin + [1.6,) + + run + + + + + false + true + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [2.8,) + + build-classpath + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + [3.2,) + + compile + + + + + + + + + + + + + + + + + org.apache.hbase + hbase-common + + + org.apache.hbase + hbase-common + test-jar + test + + + org.apache.hbase + hbase-annotations + test-jar + test + + + org.apache.hbase + hbase-protocol-shaded + + + org.apache.hbase + hbase-hadoop-compat + + + org.apache.hbase + hbase-hadoop2-compat + + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + commons-codec + commons-codec + + + org.apache.commons + commons-collections4 + + + org.codehaus.jettison + jettison + + + stax + stax-api + + + + + + com.github.stephenc.findbugs + findbugs-annotations + true + + + io.dropwizard.metrics + metrics-core + + + com.google.protobuf + protobuf-java + + + commons-cli + commons-cli + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + commons-logging + commons-logging + + + org.apache.commons + commons-math3 + + + log4j + log4j + + + org.apache.zookeeper + zookeeper + + + org.jamon + jamon-runtime + + + javax.servlet + javax.servlet-api + + + javax.ws.rs + javax.ws.rs-api + + + + + org.apache.htrace + htrace-core + + + com.lmax + disruptor + + + + org.hamcrest + hamcrest-core + test + + + org.apache.hadoop + hadoop-minikdc + test + + + org.bouncycastle + bcprov-jdk16 + test + + + org.apache.kerby + kerb-client + test + + + org.apache.kerby + kerb-simplekdc + test + + + org.apache.httpcomponents + httpclient + test + + + org.apache.httpcomponents + httpcore + test + + + org.apache.commons + commons-crypto + ${commons-crypto.version} + + + net.java.dev.jna + jna + + + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + + apache-release + + + + org.apache.maven.plugins + maven-resources-plugin + + + license-javadocs + prepare-package + + copy-resources + + + ${project.build.directory}/apidocs + + + src/main/javadoc/META-INF/ + META-INF/ + + LICENSE + NOTICE + + true + + + + + + + + + + + + skipServerTests + + + skipServerTests + + + + true + true + + + + + native + + false + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + make + compile + run + + + + + + + + + + + + + + + + + + + + + + + hadoop-2.0 + + + + !hadoop.profile + + + + + org.apache.hadoop + hadoop-distcp + ${hadoop-two.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-annotations + ${hadoop-two.version} + + + org.apache.hadoop + hadoop-client + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + test-jar + test + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-hdfs + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-hdfs + test-jar + test + + + com.google.guava + guava + + + + + org.apache.hadoop + hadoop-minicluster + test + + + com.google.guava + guava + + + + + + io.netty + netty + ${netty.hadoop.version} + test + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + + hadoop-3.0 + + + hadoop.profile + 3.0 + + + + ${hadoop-three.version} + + + + org.apache.hadoop + hadoop-distcp + ${hadoop-three.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-annotations + ${hadoop-three.version} + + + org.apache.hadoop + hadoop-minicluster + + + com.google.guava + guava + + + + + + io.netty + netty + ${netty.hadoop.version} + test + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java similarity index 88% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java index fd60765a42..1b5fac4c31 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java @@ -20,16 +20,16 @@ package org.apache.hadoop.hbase.zookeeper; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.zookeeper.KeeperException; /** * Tracker on cluster settings up in zookeeper. - * This is not related to {@link org.apache.hadoop.hbase.ClusterStatus}. That class - * is a data structure that holds snapshot of current view on cluster. This class + * This is not related to {@link org.apache.hadoop.hbase.ClusterStatus}. That class + * is a data structure that holds snapshot of current view on cluster. This class * is about tracking cluster attributes up in zookeeper. * */ @@ -66,9 +66,9 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { throws KeeperException { byte [] upData = toByteArray(); try { - ZKUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, upData); + ZooKeeperUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, upData); } catch(KeeperException.NodeExistsException nee) { - ZKUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData); + ZooKeeperUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData); } } @@ -79,7 +79,7 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { public void setClusterDown() throws KeeperException { try { - ZKUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode); + ZooKeeperUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) { LOG.warn("Attempted to set cluster as down but already down, cluster " + "state node (" + watcher.znodePaths.clusterStateZNode + ") not found"); @@ -94,6 +94,6 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { ZooKeeperProtos.ClusterUp.Builder builder = ZooKeeperProtos.ClusterUp.newBuilder(); builder.setStartDate(new java.util.Date().toString()); - return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + return ProtobufMagic.prependPBMagic(builder.build().toByteArray()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java similarity index 97% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java index c58a840f2b..31f7d0e1b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/DeletionListener.java @@ -78,7 +78,7 @@ public class DeletionListener extends ZooKeeperListener { return; } try { - if (!(ZKUtil.setWatchIfNodeExists(watcher, pathToWatch))) { + if (!(ZooKeeperUtil.setWatchIfNodeExists(watcher, pathToWatch))) { deletedLatch.countDown(); } } catch (KeeperException ex) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/EmptyWatcher.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java similarity index 98% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java index 5f3904aabc..62921f3bab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/HQuorumPeer.java @@ -71,7 +71,7 @@ public class HQuorumPeer { zkConfig.parseProperties(zkProperties); // login the zookeeper server principal (if using security) - ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE, + ZooKeeperUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE, HConstants.ZK_SERVER_KERBEROS_PRINCIPAL, zkConfig.getClientPortAddress().getHostName()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java similarity index 85% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java index 527bc17c4b..edf17f8560 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java @@ -21,10 +21,11 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.protobuf.ProtobufHelpers; +import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.LoadBalancerProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; @@ -64,9 +65,9 @@ public class LoadBalancerTracker extends ZooKeeperNodeTracker { public void setBalancerOn(boolean balancerOn) throws KeeperException { byte [] upData = toByteArray(balancerOn); try { - ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData); + ZooKeeperUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData); } catch(KeeperException.NoNodeException nne) { - ZKUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData); + ZooKeeperUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData); } super.nodeDataChanged(watcher.znodePaths.balancerZNode); } @@ -75,17 +76,17 @@ public class LoadBalancerTracker extends ZooKeeperNodeTracker { LoadBalancerProtos.LoadBalancerState.Builder builder = LoadBalancerProtos.LoadBalancerState.newBuilder(); builder.setBalancerOn(isBalancerOn); - return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + return ProtobufMagic.prependPBMagic(builder.build().toByteArray()); } private LoadBalancerProtos.LoadBalancerState parseFrom(byte [] pbBytes) throws DeserializationException { - ProtobufUtil.expectPBMagicPrefix(pbBytes); + ProtobufMagic.expectPBMagicPrefix(pbBytes); LoadBalancerProtos.LoadBalancerState.Builder builder = LoadBalancerProtos.LoadBalancerState.newBuilder(); try { - int magicLen = ProtobufUtil.lengthOfPBMagic(); - ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); + int magicLen = ProtobufMagic.lengthOfPBMagic(); + ProtobufHelpers.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); } catch (IOException e) { throw new DeserializationException(e); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java similarity index 93% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java index 7d6f9fdd91..557cabed29 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java @@ -47,8 +47,8 @@ public class MasterMaintenanceModeTracker extends ZooKeeperListener { private void update() { try { - List children = - ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.znodePaths.masterMaintZNode); + List children = ZooKeeperUtil.listChildrenAndWatchForNewChildren( + watcher, watcher.znodePaths.masterMaintZNode); hasChildren = (children != null && children.size() > 0); } catch (KeeperException e) { // Ignore the ZK keeper exception diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetricsZooKeeper.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java similarity index 100% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java similarity index 85% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java index a9939347c7..581ce69fcb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionNormalizerTracker.java @@ -22,7 +22,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.ProtobufHelpers; +import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionNormalizerProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.KeeperException; @@ -64,9 +65,9 @@ public class RegionNormalizerTracker extends ZooKeeperNodeTracker { public void setNormalizerOn(boolean normalizerOn) throws KeeperException { byte [] upData = toByteArray(normalizerOn); try { - ZKUtil.setData(watcher, watcher.znodePaths.regionNormalizerZNode, upData); + ZooKeeperUtil.setData(watcher, watcher.znodePaths.regionNormalizerZNode, upData); } catch(KeeperException.NoNodeException nne) { - ZKUtil.createAndWatch(watcher, watcher.znodePaths.regionNormalizerZNode, upData); + ZooKeeperUtil.createAndWatch(watcher, watcher.znodePaths.regionNormalizerZNode, upData); } super.nodeDataChanged(watcher.znodePaths.regionNormalizerZNode); } @@ -75,17 +76,17 @@ public class RegionNormalizerTracker extends ZooKeeperNodeTracker { RegionNormalizerProtos.RegionNormalizerState.Builder builder = RegionNormalizerProtos.RegionNormalizerState.newBuilder(); builder.setNormalizerOn(isNormalizerOn); - return ProtobufUtil.prependPBMagic(builder.build().toByteArray()); + return ProtobufMagic.prependPBMagic(builder.build().toByteArray()); } private RegionNormalizerProtos.RegionNormalizerState parseFrom(byte [] pbBytes) throws DeserializationException { - ProtobufUtil.expectPBMagicPrefix(pbBytes); + ProtobufMagic.expectPBMagicPrefix(pbBytes); RegionNormalizerProtos.RegionNormalizerState.Builder builder = RegionNormalizerProtos.RegionNormalizerState.newBuilder(); try { - int magicLen = ProtobufUtil.lengthOfPBMagic(); - ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); + int magicLen = ProtobufMagic.lengthOfPBMagic(); + ProtobufHelpers.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen); } catch (IOException e) { throw new DeserializationException(e); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java similarity index 90% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java index b0610b01ea..f8864a867c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKLeaderManager.java @@ -59,9 +59,9 @@ public class ZKLeaderManager extends ZooKeeperListener { public void start() { try { watcher.registerListener(this); - String parent = ZKUtil.getParent(leaderZNode); - if (ZKUtil.checkExists(watcher, parent) < 0) { - ZKUtil.createWithParents(watcher, parent); + String parent = ZooKeeperUtil.getParent(leaderZNode); + if (ZooKeeperUtil.checkExists(watcher, parent) < 0) { + ZooKeeperUtil.createWithParents(watcher, parent); } } catch (KeeperException ke) { watcher.abort("Unhandled zk exception when starting", ke); @@ -86,7 +86,7 @@ public class ZKLeaderManager extends ZooKeeperListener { private void handleLeaderChange() { try { synchronized(leaderExists) { - if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) { + if (ZooKeeperUtil.watchAndCheckExists(watcher, leaderZNode)) { LOG.info("Found new leader for znode: "+leaderZNode); leaderExists.set(true); } else { @@ -107,7 +107,7 @@ public class ZKLeaderManager extends ZooKeeperListener { public void waitToBecomeLeader() { while (!candidate.isStopped()) { try { - if (ZKUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) { + if (ZooKeeperUtil.createEphemeralNodeAndWatch(watcher, leaderZNode, nodeId)) { // claimed the leader znode leaderExists.set(true); if (LOG.isDebugEnabled()) { @@ -118,12 +118,12 @@ public class ZKLeaderManager extends ZooKeeperListener { } // if claiming the node failed, there should be another existing node - byte[] currentId = ZKUtil.getDataAndWatch(watcher, leaderZNode); + byte[] currentId = ZooKeeperUtil.getDataAndWatch(watcher, leaderZNode); if (currentId != null && Bytes.equals(currentId, nodeId)) { // claimed with our ID, but we didn't grab it, possibly restarted? LOG.info("Found existing leader with our ID ("+ Bytes.toStringBinary(nodeId)+"), removing"); - ZKUtil.deleteNode(watcher, leaderZNode); + ZooKeeperUtil.deleteNode(watcher, leaderZNode); leaderExists.set(false); } else { LOG.info("Found existing leader with ID: "+Bytes.toStringBinary(nodeId)); @@ -157,10 +157,10 @@ public class ZKLeaderManager extends ZooKeeperListener { if (!leaderExists.get()) { return; } - byte[] leaderId = ZKUtil.getData(watcher, leaderZNode); + byte[] leaderId = ZooKeeperUtil.getData(watcher, leaderZNode); if (leaderId != null && Bytes.equals(nodeId, leaderId)) { LOG.info("Stepping down as leader"); - ZKUtil.deleteNodeFailSilent(watcher, leaderZNode); + ZooKeeperUtil.deleteNodeFailSilent(watcher, leaderZNode); leaderExists.set(false); } else { LOG.info("Not current leader, no need to step down"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java similarity index 100% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKServerTool.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java similarity index 62% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java index 05cd8a2c22..9f7d98f8be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.zookeeper; import java.io.IOException; -import java.io.InterruptedIOException; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; import java.net.URLEncoder; @@ -30,12 +29,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.zookeeper.KeeperException; /** - * Common methods and attributes used by {@link org.apache.hadoop.hbase.master.SplitLogManager} + * Common methods and attributes used by {@link org.apache.hadoop.hbase.master.SplitLogManager} * and {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker} * running distributed splitting of WAL logs. */ @@ -50,7 +47,7 @@ public class ZKSplitLog { * @param filename log file name (only the basename) */ public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) { - return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename)); + return ZooKeeperUtil.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename)); } public static String getFileName(String node) { @@ -75,7 +72,7 @@ public class ZKSplitLog { } public static String getRescanNode(ZooKeeperWatcher zkw) { - return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN"); + return ZooKeeperUtil.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN"); } /** @@ -137,10 +134,6 @@ public class ZKSplitLog { return isCorrupt; } - /* - * Following methods come from SplitLogManager - */ - /** * check if /hbase/recovering-regions/<current region encoded name> * exists. Returns true if exists and set watcher as well. @@ -153,40 +146,25 @@ public class ZKSplitLog { isRegionMarkedRecoveringInZK(ZooKeeperWatcher zkw, String regionEncodedName) throws KeeperException { boolean result = false; - String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, regionEncodedName); + String nodePath = ZooKeeperUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, regionEncodedName); - byte[] node = ZKUtil.getDataAndWatch(zkw, nodePath); + byte[] node = ZooKeeperUtil.getDataAndWatch(zkw, nodePath); if (node != null) { result = true; } return result; } - /** - * @param bytes - Content of a failed region server or recovering region znode. - * @return long - The last flushed sequence Id for the region server - */ - public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) { - long lastRecordedFlushedSequenceId = -1l; - try { - lastRecordedFlushedSequenceId = ZKUtil.parseWALPositionFrom(bytes); - } catch (DeserializationException e) { - lastRecordedFlushedSequenceId = -1l; - LOG.warn("Can't parse last flushed sequence Id", e); - } - return lastRecordedFlushedSequenceId; - } - public static void deleteRecoveringRegionZNodes(ZooKeeperWatcher watcher, List regions) { try { if (regions == null) { // remove all children under /home/recovering-regions LOG.debug("Garbage collecting all recovering region znodes"); - ZKUtil.deleteChildrenRecursively(watcher, watcher.znodePaths.recoveringRegionsZNode); + ZooKeeperUtil.deleteChildrenRecursively(watcher, watcher.znodePaths.recoveringRegionsZNode); } else { for (String curRegion : regions) { - String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, curRegion); - ZKUtil.deleteNodeRecursively(watcher, nodePath); + String nodePath = ZooKeeperUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, curRegion); + ZooKeeperUtil.deleteNodeRecursively(watcher, nodePath); } } } catch (KeeperException e) { @@ -194,43 +172,5 @@ public class ZKSplitLog { } } - /** - * This function is used in distributedLogReplay to fetch last flushed sequence id from ZK - * @param zkw - * @param serverName - * @param encodedRegionName - * @return the last flushed sequence ids recorded in ZK of the region for serverName - * @throws IOException - */ - public static RegionStoreSequenceIds getRegionFlushedSequenceId(ZooKeeperWatcher zkw, - String serverName, String encodedRegionName) throws IOException { - // when SplitLogWorker recovers a region by directly replaying unflushed WAL edits, - // last flushed sequence Id changes when newly assigned RS flushes writes to the region. - // If the newly assigned RS fails again(a chained RS failures scenario), the last flushed - // sequence Id name space (sequence Id only valid for a particular RS instance), changes - // when different newly assigned RS flushes the region. - // 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.znodePaths.recoveringRegionsZNode, encodedRegionName); - nodePath = ZKUtil.joinZNode(nodePath, serverName); - try { - byte[] data; - try { - data = ZKUtil.getData(zkw, nodePath); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } - if (data != null) { - result = ZKUtil.parseRegionStoreSequenceIds(data); - } - } catch (KeeperException e) { - throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server=" - + serverName + "; region=" + encodedRegionName, e); - } catch (DeserializationException e) { - LOG.warn("Can't parse last flushed sequence Id from znode:" + nodePath, e); - } - return result; - } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java similarity index 79% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java index 84b53d8628..47f4df4161 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java @@ -18,11 +18,11 @@ package org.apache.hadoop.hbase.zookeeper; import static org.apache.hadoop.hbase.HConstants.DEFAULT_META_REPLICA_NUM; +import static org.apache.hadoop.hbase.HConstants.DEFAULT_REPLICA_ID; import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT; import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM; import static org.apache.hadoop.hbase.HConstants.SPLIT_LOGDIR_NAME; import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT; -import static org.apache.hadoop.hbase.HRegionInfo.DEFAULT_REPLICA_ID; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap; @@ -30,7 +30,6 @@ import java.util.Optional; import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.yetus.audience.InterfaceAudience; /** @@ -92,44 +91,44 @@ public class ZNodePaths { baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT); ImmutableMap.Builder builder = ImmutableMap.builder(); metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX); - String defaultMetaReplicaZNode = ZKUtil.joinZNode(baseZNode, metaZNodePrefix); + String defaultMetaReplicaZNode = ZooKeeperUtil.joinZNode(baseZNode, metaZNodePrefix); builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode); int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM); IntStream.range(1, numMetaReplicas) .forEachOrdered(i -> builder.put(i, defaultMetaReplicaZNode + "-" + i)); metaReplicaZNodes = builder.build(); - rsZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.rs", "rs")); - drainingZNode = ZKUtil.joinZNode(baseZNode, + rsZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.rs", "rs")); + drainingZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.draining.rs", "draining")); - masterAddressZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.master", "master")); - backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode, + masterAddressZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.master", "master")); + backupMasterAddressesZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.backup.masters", "backup-masters")); - clusterStateZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.state", "running")); - tableZNode = ZKUtil.joinZNode(baseZNode, + clusterStateZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.state", "running")); + tableZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.tableEnableDisable", "table")); - clusterIdZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.clusterId", "hbaseid")); - splitLogZNode = ZKUtil.joinZNode(baseZNode, + clusterIdZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.clusterId", "hbaseid")); + splitLogZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.splitlog", SPLIT_LOGDIR_NAME)); - balancerZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer")); - regionNormalizerZNode = ZKUtil.joinZNode(baseZNode, + balancerZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer")); + regionNormalizerZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.regionNormalizer", "normalizer")); - switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch")); - tableLockZNode = ZKUtil.joinZNode(baseZNode, + switchZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch")); + tableLockZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.tableLock", "table-lock")); - recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode, + recoveringRegionsZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.recovering.regions", "recovering-regions")); - namespaceZNode = ZKUtil.joinZNode(baseZNode, + namespaceZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.namespace", "namespace")); - masterMaintZNode = ZKUtil.joinZNode(baseZNode, + masterMaintZNode = ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.masterMaintenance", "master-maintenance")); replicationZNode = - ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.replication", "replication")); + ZooKeeperUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.replication", "replication")); peersZNode = - ZKUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.peers", "peers")); + ZooKeeperUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.peers", "peers")); queuesZNode = - ZKUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.rs", "rs")); + ZooKeeperUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.rs", "rs")); hfileRefsZNode = - ZKUtil.joinZNode(replicationZNode, + ZooKeeperUtil.joinZNode(replicationZNode, conf.get("zookeeper.znode.replication.hfile.refs", "hfile-refs")); } @@ -180,17 +179,8 @@ public class ZNodePaths { */ public int getMetaReplicaIdFromZnode(String znode) { if (znode.equals(metaZNodePrefix)) { - return HRegionInfo.DEFAULT_REPLICA_ID; + return DEFAULT_REPLICA_ID; } return Integer.parseInt(znode.substring(metaZNodePrefix.length() + 1)); } - - /** - * Is it the default meta replica's znode - * @param znode - * @return true or false - */ - public boolean isDefaultMetaReplicaZnode(String znode) { - return metaReplicaZNodes.get(DEFAULT_REPLICA_ID).equals(znode); - } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java similarity index 94% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java index 3df9880a77..e2b53dbfb6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java @@ -48,10 +48,10 @@ public class ZkAclReset extends Configured implements Tool { private static void resetAcls(final ZooKeeperWatcher zkw, final String znode, final boolean eraseAcls) throws Exception { - List children = ZKUtil.listChildrenNoWatch(zkw, znode); + List children = ZooKeeperUtil.listChildrenNoWatch(zkw, znode); if (children != null) { for (String child: children) { - resetAcls(zkw, ZKUtil.joinZNode(znode, child), eraseAcls); + resetAcls(zkw, ZooKeeperUtil.joinZNode(znode, child), eraseAcls); } } @@ -61,7 +61,7 @@ public class ZkAclReset extends Configured implements Tool { zk.setACL(znode, ZooDefs.Ids.OPEN_ACL_UNSAFE, -1); } else { LOG.info(" - set ACLs for " + znode); - zk.setACL(znode, ZKUtil.createACL(zkw, znode, true), -1); + zk.setACL(znode, ZooKeeperUtil.createACL(zkw, znode, true), -1); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperConnectionException.java similarity index 97% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperConnectionException.java index 60776dac24..2593aabd96 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ZooKeeperConnectionException.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperConnectionException.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase; +package org.apache.hadoop.hbase.zookeeper; import java.io.IOException; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperListener.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java similarity index 100% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMainServer.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java similarity index 100% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperMetricsListener.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java similarity index 92% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java index a5b084b001..50285be273 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java @@ -74,8 +74,8 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { public synchronized void start() { this.watcher.registerListener(this); try { - if(ZKUtil.watchAndCheckExists(watcher, node)) { - byte [] data = ZKUtil.getDataAndWatch(watcher, node); + if(ZooKeeperUtil.watchAndCheckExists(watcher, node)) { + byte [] data = ZooKeeperUtil.getDataAndWatch(watcher, node); if(data != null) { this.data = data; } else { @@ -123,7 +123,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { if (refresh) { try { // This does not create a watch if the node does not exists - this.data = ZKUtil.getDataAndWatch(watcher, node); + this.data = ZooKeeperUtil.getDataAndWatch(watcher, node); } catch(KeeperException e) { // We use to abort here, but in some cases the abort is ignored ( // (empty Abortable), so it's better to log... @@ -135,7 +135,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { while (!this.stopped && (notimeout || remaining > 0) && this.data == null) { if (!nodeExistsChecked) { try { - nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1); + nodeExistsChecked = (ZooKeeperUtil.checkExists(watcher, node) != -1); } catch (KeeperException e) { LOG.warn( "Got exception while trying to check existence in ZooKeeper" + @@ -147,7 +147,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { LOG.debug("Node " + node + " now exists, resetting a watcher"); try { // This does not create a watch if the node does not exists - this.data = ZKUtil.getDataAndWatch(watcher, node); + this.data = ZooKeeperUtil.getDataAndWatch(watcher, node); } catch (KeeperException e) { LOG.warn("Unexpected exception handling blockUntilAvailable", e); abortable.abort("Unexpected exception handling blockUntilAvailable", e); @@ -174,7 +174,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { public synchronized byte [] getData(boolean refresh) { if (refresh) { try { - this.data = ZKUtil.getDataAndWatch(watcher, node); + this.data = ZooKeeperUtil.getDataAndWatch(watcher, node); } catch(KeeperException e) { abortable.abort("Unexpected exception handling getData", e); } @@ -190,7 +190,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { public synchronized void nodeCreated(String path) { if (!path.equals(node)) return; try { - byte [] data = ZKUtil.getDataAndWatch(watcher, node); + byte [] data = ZooKeeperUtil.getDataAndWatch(watcher, node); if (data != null) { this.data = data; notifyAll(); @@ -206,7 +206,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { public synchronized void nodeDeleted(String path) { if(path.equals(node)) { try { - if(ZKUtil.watchAndCheckExists(watcher, node)) { + if(ZooKeeperUtil.watchAndCheckExists(watcher, node)) { nodeCreated(path); } else { this.data = null; @@ -223,7 +223,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { nodeCreated(path); } } - + /** * Checks if the baseznode set as per the property 'zookeeper.znode.parent' * exists. @@ -232,7 +232,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { */ public boolean checkIfBaseNodeAvailable() { try { - if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) { + if (ZooKeeperUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) { return false; } } catch (KeeperException e) { diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperUtil.java new file mode 100644 index 0000000000..63d2f503a0 --- /dev/null +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperUtil.java @@ -0,0 +1,1886 @@ +package org.apache.hadoop.hbase.zookeeper; + + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.PrintWriter; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Deque; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.AuthUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.protobuf.ProtobufMagic; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.security.Superusers; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.util.Addressing; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp.CreateAndFailSilent; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp.DeleteNodeFailSilent; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperUtil.ZKUtilOp.SetData; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooDefs.Perms; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZooKeeperSaslClient; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.CreateRequest; +import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.SetDataRequest; +import org.apache.zookeeper.server.ZooKeeperSaslServer; + +public class ZooKeeperUtil { + private static final Log LOG = LogFactory.getLog(ZooKeeperUtil.class); + // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved. + public static final char ZNODE_PATH_SEPARATOR = '/'; + + /** + * Creates a new connection to ZooKeeper, pulling settings and ensemble config + * from the specified configuration object using methods from {@link ZKConfig}. + * + * Sets the connection status monitoring watcher to the specified watcher. + * + * @param conf configuration to pull ensemble and other settings from + * @param watcher watcher to monitor connection changes + * @return connection to zookeeper + * @throws IOException if unable to connect to zk or config problem + */ + public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher) + throws IOException { + String ensemble = ZKConfig.getZKQuorumServersString(conf); + return connect(conf, ensemble, watcher); + } + + public static RecoverableZooKeeper connect(Configuration conf, String ensemble, + Watcher watcher) + throws IOException { + return connect(conf, ensemble, watcher, null); + } + + public static RecoverableZooKeeper connect(Configuration conf, String ensemble, + Watcher watcher, final String identifier) + throws IOException { + if(ensemble == null) { + throw new IOException("Unable to determine ZooKeeper ensemble"); + } + int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, + HConstants.DEFAULT_ZK_SESSION_TIMEOUT); + if (LOG.isTraceEnabled()) { + LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble); + } + int retry = conf.getInt("zookeeper.recovery.retry", 3); + int retryIntervalMillis = + conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); + int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000); + return new RecoverableZooKeeper(ensemble, timeout, watcher, + retry, retryIntervalMillis, maxSleepTime, identifier); + } + + /** + * Log in the current zookeeper server process using the given configuration + * keys for the credential file and login principal. + * + *

This is only applicable when running on secure hbase + * On regular HBase (without security features), this will safely be ignored. + *

+ * + * @param conf The configuration data to use + * @param keytabFileKey Property key used to configure the path to the credential file + * @param userNameKey Property key used to configure the login principal + * @param hostname Current hostname to use in any credentials + * @throws IOException underlying exception from SecurityUtil.login() call + */ + public static void loginServer(Configuration conf, String keytabFileKey, + String userNameKey, String hostname) throws IOException { + login(conf, keytabFileKey, userNameKey, hostname, + ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY, + JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME); + } + + /** + * Log in the current zookeeper client using the given configuration + * keys for the credential file and login principal. + * + *

This is only applicable when running on secure hbase + * On regular HBase (without security features), this will safely be ignored. + *

+ * + * @param conf The configuration data to use + * @param keytabFileKey Property key used to configure the path to the credential file + * @param userNameKey Property key used to configure the login principal + * @param hostname Current hostname to use in any credentials + * @throws IOException underlying exception from SecurityUtil.login() call + */ + public static void loginClient(Configuration conf, String keytabFileKey, + String userNameKey, String hostname) throws IOException { + login(conf, keytabFileKey, userNameKey, hostname, + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, + JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME); + } + + /** + * Log in the current process using the given configuration keys for the + * credential file and login principal. + * + *

This is only applicable when running on secure hbase + * On regular HBase (without security features), this will safely be ignored. + *

+ * + * @param conf The configuration data to use + * @param keytabFileKey Property key used to configure the path to the credential file + * @param userNameKey Property key used to configure the login principal + * @param hostname Current hostname to use in any credentials + * @param loginContextProperty property name to expose the entry name + * @param loginContextName jaas entry name + * @throws IOException underlying exception from SecurityUtil.login() call + */ + private static void login(Configuration conf, String keytabFileKey, + String userNameKey, String hostname, + String loginContextProperty, String loginContextName) + throws IOException { + if (!isSecureZooKeeper(conf)) + return; + + // User has specified a jaas.conf, keep this one as the good one. + // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf" + if (System.getProperty("java.security.auth.login.config") != null) + return; + + // No keytab specified, no auth + String keytabFilename = conf.get(keytabFileKey); + if (keytabFilename == null) { + LOG.warn("no keytab specified for: " + keytabFileKey); + return; + } + + String principalConfig = conf.get(userNameKey, System.getProperty("user.name")); + String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname); + + // Initialize the "jaas.conf" for keyTab/principal, + // If keyTab is not specified use the Ticket Cache. + // and set the zookeeper login context name. + JaasConfiguration jaasConf = new JaasConfiguration(loginContextName, + principalName, keytabFilename); + javax.security.auth.login.Configuration.setConfiguration(jaasConf); + System.setProperty(loginContextProperty, loginContextName); + } + + /** + * A JAAS configuration that defines the login modules that we want to use for login. + */ + private static class JaasConfiguration extends javax.security.auth.login.Configuration { + private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME = + "zookeeper-server-keytab-kerberos"; + private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME = + "zookeeper-client-keytab-kerberos"; + + private static final Map BASIC_JAAS_OPTIONS = new HashMap<>(); + static { + String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG"); + if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { + BASIC_JAAS_OPTIONS.put("debug", "true"); + } + } + + private static final Map KEYTAB_KERBEROS_OPTIONS = new HashMap<>(); + static { + KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true"); + KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true"); + KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true"); + KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS); + } + + private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN = + new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(), + LoginModuleControlFlag.REQUIRED, + KEYTAB_KERBEROS_OPTIONS); + + private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF = + new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN}; + + private javax.security.auth.login.Configuration baseConfig; + private final String loginContextName; + private final boolean useTicketCache; + private final String keytabFile; + private final String principal; + + public JaasConfiguration(String loginContextName, String principal, String keytabFile) { + this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0); + } + + private JaasConfiguration(String loginContextName, String principal, + String keytabFile, boolean useTicketCache) { + try { + this.baseConfig = javax.security.auth.login.Configuration.getConfiguration(); + } catch (SecurityException e) { + this.baseConfig = null; + } + this.loginContextName = loginContextName; + this.useTicketCache = useTicketCache; + this.keytabFile = keytabFile; + this.principal = principal; + LOG.info("JaasConfiguration loginContextName=" + loginContextName + + " principal=" + principal + " useTicketCache=" + useTicketCache + + " keytabFile=" + keytabFile); + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String appName) { + if (loginContextName.equals(appName)) { + if (!useTicketCache) { + KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile); + KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true"); + } + KEYTAB_KERBEROS_OPTIONS.put("principal", principal); + KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false"); + return KEYTAB_KERBEROS_CONF; + } + if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName); + return(null); + } + } + + // + // Helper methods + // + + /** + * Join the prefix znode name with the suffix znode name to generate a proper + * full znode name. + * + * Assumes prefix does not end with slash and suffix does not begin with it. + * + * @param prefix beginning of znode name + * @param suffix ending of znode name + * @return result of properly joining prefix with suffix + */ + public static String joinZNode(String prefix, String suffix) { + return prefix + ZNODE_PATH_SEPARATOR + suffix; + } + + /** + * Returns the full path of the immediate parent of the specified node. + * @param node path to get parent of + * @return parent of path, null if passed the root node or an invalid node + */ + public static String getParent(String node) { + int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR); + return idx <= 0 ? null : node.substring(0, idx); + } + + /** + * Get the name of the current node from the specified fully-qualified path. + * @param path fully-qualified path + * @return name of the current node + */ + public static String getNodeName(String path) { + return path.substring(path.lastIndexOf("/")+1); + } + + // + // Existence checks and watches + // + + /** + * Watch the specified znode for delete/create/change events. The watcher is + * set whether or not the node exists. If the node already exists, the method + * returns true. If the node does not exist, the method returns false. + * + * @param zkw zk reference + * @param znode path of node to watch + * @return true if znode exists, false if does not exist or error + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw); + boolean exists = s != null ? true : false; + if (exists) { + LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode)); + } else { + LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode)); + } + return exists; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + zkw.keeperException(e); + return false; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + zkw.interruptedException(e); + return false; + } + } + + /** + * Watch the specified znode, but only if exists. Useful when watching + * for deletions. Uses .getData() (and handles NoNodeException) instead + * of .exists() to accomplish this, as .getData() will only set a watch if + * the znode exists. + * @param zkw zk reference + * @param znode path of node to watch + * @return true if the watch is set, false if node does not exists + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + zkw.getRecoverableZooKeeper().getData(znode, true, null); + return true; + } catch (NoNodeException e) { + return false; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + zkw.interruptedException(e); + return false; + } + } + + /** + * Check if the specified node exists. Sets no watches. + * + * @param zkw zk reference + * @param znode path of node to watch + * @return version of the node if it exists, -1 if does not exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static int checkExists(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + Stat s = zkw.getRecoverableZooKeeper().exists(znode, null); + return s != null ? s.getVersion() : -1; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); + zkw.keeperException(e); + return -1; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); + zkw.interruptedException(e); + return -1; + } + } + + // + // Znode listings + // + + /** + * Lists the children znodes of the specified znode. Also sets a watch on + * the specified znode which will capture a NodeDeleted event on the specified + * znode as well as NodeChildrenChanged if any children of the specified znode + * are created or deleted. + * + * Returns null if the specified node does not exist. Otherwise returns a + * list of children of the specified node. If the node exists but it has no + * children, an empty list will be returned. + * + * @param zkw zk reference + * @param znode path of node to list and watch children of + * @return list of children of the specified node, an empty list if the node + * exists but has no children, and null if the node does not exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static List listChildrenAndWatchForNewChildren( + ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + List children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw); + return children; + } catch(KeeperException.NoNodeException ke) { + LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + + "because node does not exist (not an error)")); + return null; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); + zkw.keeperException(e); + return null; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); + zkw.interruptedException(e); + return null; + } + } + + /** + * List all the children of the specified znode, setting a watch for children + * changes and also setting a watch on every individual child in order to get + * the NodeCreated and NodeDeleted events. + * @param zkw zookeeper reference + * @param znode node to get children of and watch + * @return list of znode names, null if the node doesn't exist + * @throws KeeperException + */ + public static List listChildrenAndWatchThem(ZooKeeperWatcher zkw, + String znode) throws KeeperException { + List children = listChildrenAndWatchForNewChildren(zkw, znode); + if (children == null) { + return null; + } + for (String child : children) { + watchAndCheckExists(zkw, joinZNode(znode, child)); + } + return children; + } + + /** + * Lists the children of the specified znode without setting any watches. + * + * Sets no watches at all, this method is best effort. + * + * Returns an empty list if the node has no children. Returns null if the + * parent node itself does not exist. + * + * @param zkw zookeeper reference + * @param znode node to get children + * @return list of data of children of specified znode, empty if no children, + * null if parent does not exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static List listChildrenNoWatch(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + List children = null; + try { + // List the children without watching + children = zkw.getRecoverableZooKeeper().getChildren(znode, null); + } catch(KeeperException.NoNodeException nne) { + return null; + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + return children; + } + + /** + * Simple class to hold a node path and node data. + * @deprecated Unused + */ + @Deprecated + public static class NodeAndData { + private String node; + private byte [] data; + public NodeAndData(String node, byte [] data) { + this.node = node; + this.data = data; + } + public String getNode() { + return node; + } + public byte [] getData() { + return data; + } + @Override + public String toString() { + return node; + } + public boolean isEmpty() { + return (data == null || data.length == 0); + } + } + + /** + * Checks if the specified znode has any children. Sets no watches. + * + * Returns true if the node exists and has children. Returns false if the + * node does not exist or if the node does not have any children. + * + * Used during master initialization to determine if the master is a + * failed-over-to master or the first master during initial cluster startup. + * If the directory for regionserver ephemeral nodes is empty then this is + * a cluster startup, if not then it is not cluster startup. + * + * @param zkw zk reference + * @param znode path of node to check for children of + * @return true if node has children, false if not or node does not exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty(); + } catch(KeeperException.NoNodeException ke) { + LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + + "because node does not exist (not an error)")); + return false; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); + zkw.keeperException(e); + return false; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); + zkw.interruptedException(e); + return false; + } + } + + /** + * Get the number of children of the specified node. + * + * If the node does not exist or has no children, returns 0. + * + * Sets no watches at all. + * + * @param zkw zk reference + * @param znode path of node to count children of + * @return number of children of specified node, 0 if none or parent does not + * exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + try { + Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null); + return stat == null ? 0 : stat.getNumChildren(); + } catch(KeeperException e) { + LOG.warn(zkw.prefix("Unable to get children of node " + znode)); + zkw.keeperException(e); + } catch(InterruptedException e) { + zkw.interruptedException(e); + } + return 0; + } + + // + // Data retrieval + // + + /** + * Get znode data. Does not set a watcher. + * @return ZNode data, null if the node does not exist or if there is an + * error. + */ + public static byte [] getData(ZooKeeperWatcher zkw, String znode) + throws KeeperException, InterruptedException { + try { + byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null); + logRetrievedMsg(zkw, znode, data, false); + return data; + } catch (KeeperException.NoNodeException e) { + LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + + "because node does not exist (not an error)")); + return null; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + zkw.keeperException(e); + return null; + } + } + + /** + * Get the data at the specified znode and set a watch. + * + * Returns the data and sets a watch if the node exists. Returns null and no + * watch is set if the node does not exist or there is an exception. + * + * @param zkw zk reference + * @param znode path of node + * @return data of the specified znode, or null + * @throws KeeperException if unexpected zookeeper exception + */ + public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + return getDataInternal(zkw, znode, null, true); + } + + /** + * Get the data at the specified znode and set a watch. + * + * Returns the data and sets a watch if the node exists. Returns null and no + * watch is set if the node does not exist or there is an exception. + * + * @param zkw zk reference + * @param znode path of node + * @param stat object to populate the version of the znode + * @return data of the specified znode, or null + * @throws KeeperException if unexpected zookeeper exception + */ + public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode, + Stat stat) throws KeeperException { + return getDataInternal(zkw, znode, stat, true); + } + + private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat, + boolean watcherSet) + throws KeeperException { + try { + byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat); + logRetrievedMsg(zkw, znode, data, watcherSet); + return data; + } catch (KeeperException.NoNodeException e) { + // This log can get pretty annoying when we cycle on 100ms waits. + // Enable trace if you really want to see it. + LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " + + "because node does not exist (not an error)")); + return null; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + zkw.keeperException(e); + return null; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + zkw.interruptedException(e); + return null; + } + } + + /** + * Get the data at the specified znode without setting a watch. + * + * Returns the data if the node exists. Returns null if the node does not + * exist. + * + * Sets the stats of the node in the passed Stat object. Pass a null stat if + * not interested. + * + * @param zkw zk reference + * @param znode path of node + * @param stat node status to get if node exists + * @return data of the specified znode, or null if node does not exist + * @throws KeeperException if unexpected zookeeper exception + */ + public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode, + Stat stat) + throws KeeperException { + try { + byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat); + logRetrievedMsg(zkw, znode, data, false); + return data; + } catch (KeeperException.NoNodeException e) { + LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + + "because node does not exist (not necessarily an error)")); + return null; + } catch (KeeperException e) { + LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + zkw.keeperException(e); + return null; + } catch (InterruptedException e) { + LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + zkw.interruptedException(e); + return null; + } + } + + /** + * Returns the date of child znodes of the specified znode. Also sets a watch on + * the specified znode which will capture a NodeDeleted event on the specified + * znode as well as NodeChildrenChanged if any children of the specified znode + * are created or deleted. + * + * Returns null if the specified node does not exist. Otherwise returns a + * list of children of the specified node. If the node exists but it has no + * children, an empty list will be returned. + * + * @param zkw zk reference + * @param baseNode path of node to list and watch children of + * @return list of data of children of the specified node, an empty list if the node + * exists but has no children, and null if the node does not exist + * @throws KeeperException if unexpected zookeeper exception + * @deprecated Unused + */ + @Deprecated + public static List getChildDataAndWatchForNewChildren( + ZooKeeperWatcher zkw, String baseNode) throws KeeperException { + List nodes = listChildrenAndWatchForNewChildren(zkw, baseNode); + if (nodes != null) { + List newNodes = new ArrayList<>(); + for (String node : nodes) { + String nodePath = joinZNode(baseNode, node); + byte[] data = getDataAndWatch(zkw, nodePath); + newNodes.add(new NodeAndData(nodePath, data)); + } + return newNodes; + } + return null; + } + + /** + * Update the data of an existing node with the expected version to have the + * specified data. + * + * Throws an exception if there is a version mismatch or some other problem. + * + * Sets no watches under any conditions. + * + * @param zkw zk reference + * @param znode + * @param data + * @param expectedVersion + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.BadVersionException if version mismatch + * @deprecated Unused + */ + @Deprecated + public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode, + byte [] data, int expectedVersion) + throws KeeperException { + try { + zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion); + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + // + // Data setting + // + + /** + * Sets the data of the existing znode to be the specified data. Ensures that + * the current data has the specified expected version. + * + *

If the node does not exist, a {@link NoNodeException} will be thrown. + * + *

If their is a version mismatch, method returns null. + * + *

No watches are set but setting data will trigger other watchers of this + * node. + * + *

If there is another problem, a KeeperException will be thrown. + * + * @param zkw zk reference + * @param znode path of node + * @param data data to set for node + * @param expectedVersion version expected when setting data + * @return true if data set, false if version mismatch + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean setData(ZooKeeperWatcher zkw, String znode, + byte [] data, int expectedVersion) + throws KeeperException, KeeperException.NoNodeException { + try { + return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null; + } catch (InterruptedException e) { + zkw.interruptedException(e); + return false; + } + } + + /** + * Set data into node creating node if it doesn't yet exist. + * Does not set watch. + * + * @param zkw zk reference + * @param znode path of node + * @param data data to set for node + * @throws KeeperException + */ + public static void createSetData(final ZooKeeperWatcher zkw, final String znode, + final byte [] data) + throws KeeperException { + if (checkExists(zkw, znode) == -1) { + createWithParents(zkw, znode, data); + } else { + setData(zkw, znode, data); + } + } + + /** + * Sets the data of the existing znode to be the specified data. The node + * must exist but no checks are done on the existing data or version. + * + *

If the node does not exist, a {@link NoNodeException} will be thrown. + * + *

No watches are set but setting data will trigger other watchers of this + * node. + * + *

If there is another problem, a KeeperException will be thrown. + * + * @param zkw zk reference + * @param znode path of node + * @param data data to set for node + * @throws KeeperException if unexpected zookeeper exception + */ + public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data) + throws KeeperException { + setData(zkw, (SetData)ZKUtilOp.setData(znode, data)); + } + + private static void setData(ZooKeeperWatcher zkw, SetData setData) + throws KeeperException { + SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord(); + setData(zkw, sd.getPath(), sd.getData(), sd.getVersion()); + } + + /** + * Returns whether or not secure authentication is enabled + * (whether hbase.security.authentication is set to + * kerberos. + */ + public static boolean isSecureZooKeeper(Configuration conf) { + // Detection for embedded HBase client with jaas configuration + // defined for third party programs. + try { + javax.security.auth.login.Configuration testConfig = + javax.security.auth.login.Configuration.getConfiguration(); + if (testConfig.getAppConfigurationEntry("Client") == null + && testConfig.getAppConfigurationEntry( + JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null + && testConfig.getAppConfigurationEntry( + JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null + && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null + && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) { + + return false; + } + } catch(Exception e) { + // No Jaas configuration defined. + return false; + } + + // Master & RSs uses hbase.zookeeper.client.* + return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication")); + } + + private static ArrayList createACL(ZooKeeperWatcher zkw, String node) { + return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration())); + } + + public static ArrayList createACL(ZooKeeperWatcher zkw, String node, + boolean isSecureZooKeeper) { + if (!node.startsWith(zkw.znodePaths.baseZNode)) { + return Ids.OPEN_ACL_UNSAFE; + } + if (isSecureZooKeeper) { + ArrayList acls = new ArrayList<>(); + // add permission to hbase supper user + String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY); + String hbaseUser = null; + try { + hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + LOG.debug("Could not acquire current User.", e); + } + if (superUsers != null) { + List groups = new ArrayList<>(); + for (String user : superUsers) { + if (AuthUtil.isGroupPrincipal(user)) { + // TODO: Set node ACL for groups when ZK supports this feature + groups.add(user); + } else { + if(!user.equals(hbaseUser)) { + acls.add(new ACL(Perms.ALL, new Id("sasl", user))); + } + } + } + if (!groups.isEmpty()) { + LOG.warn("Znode ACL setting for group " + groups + + " is skipped, ZooKeeper doesn't support this feature presently."); + } + } + // Certain znodes are accessed directly by the client, + // so they must be readable by non-authenticated clients + if (zkw.isClientReadable(node)) { + acls.addAll(Ids.CREATOR_ALL_ACL); + acls.addAll(Ids.READ_ACL_UNSAFE); + } else { + acls.addAll(Ids.CREATOR_ALL_ACL); + } + return acls; + } else { + return Ids.OPEN_ACL_UNSAFE; + } + } + + // + // Node creation + // + + /** + * + * Set the specified znode to be an ephemeral node carrying the specified + * data. + * + * If the node is created successfully, a watcher is also set on the node. + * + * If the node is not created successfully because it already exists, this + * method will also set a watcher on the node. + * + * If there is another problem, a KeeperException will be thrown. + * + * @param zkw zk reference + * @param znode path of node + * @param data data of node + * @return true if node created, false if not, watch set in both cases + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw, + String znode, byte [] data) + throws KeeperException { + boolean ret = true; + try { + zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), + CreateMode.EPHEMERAL); + } catch (KeeperException.NodeExistsException nee) { + ret = false; + } catch (InterruptedException e) { + LOG.info("Interrupted", e); + Thread.currentThread().interrupt(); + } + if(!watchAndCheckExists(zkw, znode)) { + // It did exist but now it doesn't, try again + return createEphemeralNodeAndWatch(zkw, znode, data); + } + return ret; + } + + /** + * Creates the specified znode to be a persistent node carrying the specified + * data. + * + * Returns true if the node was successfully created, false if the node + * already existed. + * + * If the node is created successfully, a watcher is also set on the node. + * + * If the node is not created successfully because it already exists, this + * method will also set a watcher on the node but return false. + * + * If there is another problem, a KeeperException will be thrown. + * + * @param zkw zk reference + * @param znode path of node + * @param data data of node + * @return true if node created, false if not, watch set in both cases + * @throws KeeperException if unexpected zookeeper exception + */ + public static boolean createNodeIfNotExistsAndWatch( + ZooKeeperWatcher zkw, String znode, byte [] data) + throws KeeperException { + boolean ret = true; + try { + zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), + CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException nee) { + ret = false; + } catch (InterruptedException e) { + zkw.interruptedException(e); + return false; + } + try { + zkw.getRecoverableZooKeeper().exists(znode, zkw); + } catch (InterruptedException e) { + zkw.interruptedException(e); + return false; + } + return ret; + } + + /** + * Creates the specified znode with the specified data but does not watch it. + * + * Returns the znode of the newly created node + * + * If there is another problem, a KeeperException will be thrown. + * + * @param zkw zk reference + * @param znode path of node + * @param data data of node + * @param createMode specifying whether the node to be created is ephemeral and/or sequential + * @return true name of the newly created znode or null + * @throws KeeperException if unexpected zookeeper exception + */ + public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode, + byte[] data, CreateMode createMode) throws KeeperException { + + String createdZNode = null; + try { + createdZNode = zkw.getRecoverableZooKeeper().create(znode, data, + createACL(zkw, znode), createMode); + } catch (KeeperException.NodeExistsException nee) { + return znode; + } catch (InterruptedException e) { + zkw.interruptedException(e); + return null; + } + return createdZNode; + } + + /** + * Creates the specified node with the specified data and watches it. + * + *

Throws an exception if the node already exists. + * + *

The node created is persistent and open access. + * + *

Returns the version number of the created node if successful. + * + * @param zkw zk reference + * @param znode path of node to create + * @param data data of node to create + * @return version of node created + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NodeExistsException if node already exists + */ + public static int createAndWatch(ZooKeeperWatcher zkw, + String znode, byte [] data) + throws KeeperException, KeeperException.NodeExistsException { + try { + zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), + CreateMode.PERSISTENT); + Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw); + if (stat == null){ + // Likely a race condition. Someone deleted the znode. + throw KeeperException.create(KeeperException.Code.SYSTEMERROR, + "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode); + } + return stat.getVersion(); + } catch (InterruptedException e) { + zkw.interruptedException(e); + return -1; + } + } + + /** + * Async creates the specified node with the specified data. + * + *

Throws an exception if the node already exists. + * + *

The node created is persistent and open access. + * + * @param zkw zk reference + * @param znode path of node to create + * @param data data of node to create + * @param cb + * @param ctx + */ + public static void asyncCreate(ZooKeeperWatcher zkw, + String znode, byte [] data, final AsyncCallback.StringCallback cb, + final Object ctx) { + zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data, + createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx); + } + + /** + * Creates the specified node, iff the node does not exist. Does not set a + * watch and fails silently if the node already exists. + * + * The node created is persistent and open access. + * + * @param zkw zk reference + * @param znode path of node + * @throws KeeperException if unexpected zookeeper exception + */ + public static void createAndFailSilent(ZooKeeperWatcher zkw, + String znode) throws KeeperException { + createAndFailSilent(zkw, znode, new byte[0]); + } + + /** + * Creates the specified node containing specified data, iff the node does not exist. Does + * not set a watch and fails silently if the node already exists. + * + * The node created is persistent and open access. + * + * @param zkw zk reference + * @param znode path of node + * @param data a byte array data to store in the znode + * @throws KeeperException if unexpected zookeeper exception + */ + public static void createAndFailSilent(ZooKeeperWatcher zkw, + String znode, byte[] data) + throws KeeperException { + createAndFailSilent(zkw, + (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data)); + } + + private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs) + throws KeeperException { + CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord(); + String znode = create.getPath(); + try { + RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper(); + if (zk.exists(znode, false) == null) { + zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags())); + } + } catch(KeeperException.NodeExistsException nee) { + } catch(KeeperException.NoAuthException nee){ + try { + if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) { + // If we failed to create the file and it does not already exist. + throw(nee); + } + } catch (InterruptedException ie) { + zkw.interruptedException(ie); + } + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + /** + * Creates the specified node and all parent nodes required for it to exist. + * + * No watches are set and no errors are thrown if the node already exists. + * + * The nodes created are persistent and open access. + * + * @param zkw zk reference + * @param znode path of node + * @throws KeeperException if unexpected zookeeper exception + */ + public static void createWithParents(ZooKeeperWatcher zkw, String znode) + throws KeeperException { + createWithParents(zkw, znode, new byte[0]); + } + + /** + * Creates the specified node and all parent nodes required for it to exist. The creation of + * parent znodes is not atomic with the leafe znode creation but the data is written atomically + * when the leaf node is created. + * + * No watches are set and no errors are thrown if the node already exists. + * + * The nodes created are persistent and open access. + * + * @param zkw zk reference + * @param znode path of node + * @throws KeeperException if unexpected zookeeper exception + */ + public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data) + throws KeeperException { + try { + if(znode == null) { + return; + } + zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), + CreateMode.PERSISTENT); + } catch(KeeperException.NodeExistsException nee) { + return; + } catch(KeeperException.NoNodeException nne) { + createWithParents(zkw, getParent(znode)); + createWithParents(zkw, znode, data); + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + // + // Deletes + // + + /** + * Delete the specified node. Sets no watches. Throws all exceptions. + */ + public static void deleteNode(ZooKeeperWatcher zkw, String node) + throws KeeperException { + deleteNode(zkw, node, -1); + } + + /** + * Delete the specified node with the specified version. Sets no watches. + * Throws all exceptions. + */ + public static boolean deleteNode(ZooKeeperWatcher zkw, String node, + int version) + throws KeeperException { + try { + zkw.getRecoverableZooKeeper().delete(node, version); + return true; + } catch(KeeperException.BadVersionException bve) { + return false; + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + return false; + } + } + + /** + * Deletes the specified node. Fails silent if the node does not exist. + * @param zkw + * @param node + * @throws KeeperException + */ + public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node) + throws KeeperException { + deleteNodeFailSilent(zkw, + (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node)); + } + + private static void deleteNodeFailSilent(ZooKeeperWatcher zkw, + DeleteNodeFailSilent dnfs) throws KeeperException { + DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord(); + try { + zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion()); + } catch(KeeperException.NoNodeException nne) { + } catch(InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + + /** + * Delete the specified node and all of it's children. + *

+ * If the node does not exist, just returns. + *

+ * Sets no watches. Throws all exceptions besides dealing with deletion of + * children. + */ + public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node) + throws KeeperException { + deleteNodeRecursivelyMultiOrSequential(zkw, true, node); + } + + /** + * Delete all the children of the specified node but not the node itself. + * + * Sets no watches. Throws all exceptions besides dealing with deletion of + * children. + * + * @throws KeeperException + */ + public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node) + throws KeeperException { + deleteChildrenRecursivelyMultiOrSequential(zkw, true, node); + } + + /** + * Delete all the children of the specified node but not the node itself. This + * will first traverse the znode tree for listing the children and then delete + * these znodes using multi-update api or sequential based on the specified + * configurations. + *

+ * Sets no watches. Throws all exceptions besides dealing with deletion of + * children. + *

+ * If the following is true: + *

    + *
  • runSequentialOnMultiFailure is true + *
+ * on calling multi, we get a ZooKeeper exception that can be handled by a + * sequential call(*), we retry the operations one-by-one (sequentially). + * + * @param zkw + * - zk reference + * @param runSequentialOnMultiFailure + * - if true when we get a ZooKeeper exception that could retry the + * operations one-by-one (sequentially) + * @param pathRoots + * - path of the parent node(s) + * @throws KeeperException.NotEmptyException + * if node has children while deleting + * @throws KeeperException + * if unexpected ZooKeeper exception + * @throws IllegalArgumentException + * if an invalid path is specified + */ + public static void deleteChildrenRecursivelyMultiOrSequential( + ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure, + String... pathRoots) throws KeeperException { + if (pathRoots == null || pathRoots.length <= 0) { + LOG.warn("Given path is not valid!"); + return; + } + List ops = new ArrayList<>(); + for (String eachRoot : pathRoots) { + List children = listChildrenBFSNoWatch(zkw, eachRoot); + // Delete the leaves first and eventually get rid of the root + for (int i = children.size() - 1; i >= 0; --i) { + ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i))); + } + } + // atleast one element should exist + if (ops.size() > 0) { + multiOrSequential(zkw, ops, runSequentialOnMultiFailure); + } + } + + /** + * Delete the specified node and its children. This traverse the + * znode tree for listing the children and then delete + * these znodes including the parent using multi-update api or + * sequential based on the specified configurations. + *

+ * Sets no watches. Throws all exceptions besides dealing with deletion of + * children. + *

+ * If the following is true: + *

    + *
  • runSequentialOnMultiFailure is true + *
+ * on calling multi, we get a ZooKeeper exception that can be handled by a + * sequential call(*), we retry the operations one-by-one (sequentially). + * + * @param zkw + * - zk reference + * @param runSequentialOnMultiFailure + * - if true when we get a ZooKeeper exception that could retry the + * operations one-by-one (sequentially) + * @param pathRoots + * - path of the parent node(s) + * @throws KeeperException.NotEmptyException + * if node has children while deleting + * @throws KeeperException + * if unexpected ZooKeeper exception + * @throws IllegalArgumentException + * if an invalid path is specified + */ + public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw, + boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException { + if (pathRoots == null || pathRoots.length <= 0) { + LOG.warn("Given path is not valid!"); + return; + } + List ops = new ArrayList<>(); + for (String eachRoot : pathRoots) { + // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted + // recursively, must set another watch, get notified of delete node + List children = listChildrenBFSAndWatchThem(zkw, eachRoot); + // Delete the leaves first and eventually get rid of the root + for (int i = children.size() - 1; i >= 0; --i) { + ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i))); + } + try { + if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) { + ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot)); + } + } catch (InterruptedException e) { + zkw.interruptedException(e); + } + } + // atleast one element should exist + if (ops.size() > 0) { + multiOrSequential(zkw, ops, runSequentialOnMultiFailure); + } + } + + /** + * BFS Traversal of all the children under path, with the entries in the list, + * in the same order as that of the traversal. Lists all the children without + * setting any watches. + * + * @param zkw + * - zk reference + * @param znode + * - path of node + * @return list of children znodes under the path + * @throws KeeperException + * if unexpected ZooKeeper exception + */ + private static List listChildrenBFSNoWatch(ZooKeeperWatcher zkw, + final String znode) throws KeeperException { + Deque queue = new LinkedList<>(); + List tree = new ArrayList<>(); + queue.add(znode); + while (true) { + String node = queue.pollFirst(); + if (node == null) { + break; + } + List children = listChildrenNoWatch(zkw, node); + if (children == null) { + continue; + } + for (final String child : children) { + final String childPath = node + "/" + child; + queue.add(childPath); + tree.add(childPath); + } + } + return tree; + } + + /** + * BFS Traversal of all the children under path, with the entries in the list, + * in the same order as that of the traversal. + * Lists all the children and set watches on to them. + * + * @param zkw + * - zk reference + * @param znode + * - path of node + * @return list of children znodes under the path + * @throws KeeperException + * if unexpected ZooKeeper exception + */ + private static List listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode) + throws KeeperException { + Deque queue = new LinkedList<>(); + List tree = new ArrayList<>(); + queue.add(znode); + while (true) { + String node = queue.pollFirst(); + if (node == null) { + break; + } + List children = listChildrenAndWatchThem(zkw, node); + if (children == null) { + continue; + } + for (final String child : children) { + final String childPath = node + "/" + child; + queue.add(childPath); + tree.add(childPath); + } + } + return tree; + } + + /** + * Represents an action taken by ZKUtil, e.g. createAndFailSilent. + * These actions are higher-level than ZKOp actions, which represent + * individual actions in the ZooKeeper API, like create. + */ + public abstract static class ZKUtilOp { + private String path; + + private ZKUtilOp(String path) { + this.path = path; + } + + /** + * @return a createAndFailSilent ZKUtilOp + */ + public static ZKUtilOp createAndFailSilent(String path, byte[] data) { + return new CreateAndFailSilent(path, data); + } + + /** + * @return a deleteNodeFailSilent ZKUtilOP + */ + public static ZKUtilOp deleteNodeFailSilent(String path) { + return new DeleteNodeFailSilent(path); + } + + /** + * @return a setData ZKUtilOp + */ + public static ZKUtilOp setData(String path, byte [] data) { + return new SetData(path, data); + } + + /** + * @return path to znode where the ZKOp will occur + */ + public String getPath() { + return path; + } + + /** + * ZKUtilOp representing createAndFailSilent in ZooKeeper + * (attempt to create node, ignore error if already exists) + */ + public static class CreateAndFailSilent extends ZKUtilOp { + private byte [] data; + + private CreateAndFailSilent(String path, byte [] data) { + super(path); + this.data = data; + } + + public byte[] getData() { + return data; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof CreateAndFailSilent)) return false; + + CreateAndFailSilent op = (CreateAndFailSilent) o; + return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); + } + + @Override + public int hashCode() { + int ret = 17 + getPath().hashCode() * 31; + return ret * 31 + Bytes.hashCode(data); + } + } + + /** + * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper + * (attempt to delete node, ignore error if node doesn't exist) + */ + public static class DeleteNodeFailSilent extends ZKUtilOp { + private DeleteNodeFailSilent(String path) { + super(path); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof DeleteNodeFailSilent)) return false; + + return super.equals(o); + } + + @Override + public int hashCode() { + return getPath().hashCode(); + } + } + + /** + * ZKUtilOp representing setData in ZooKeeper + */ + public static class SetData extends ZKUtilOp { + private byte [] data; + + private SetData(String path, byte [] data) { + super(path); + this.data = data; + } + + public byte[] getData() { + return data; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof SetData)) return false; + + SetData op = (SetData) o; + return getPath().equals(op.getPath()) && Arrays.equals(data, op.data); + } + + @Override + public int hashCode() { + int ret = getPath().hashCode(); + return ret * 31 + Bytes.hashCode(data); + } + } + } + + /** + * Convert from ZKUtilOp to ZKOp + */ + private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op) + throws UnsupportedOperationException { + if(op == null) return null; + + if (op instanceof CreateAndFailSilent) { + CreateAndFailSilent cafs = (CreateAndFailSilent)op; + return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()), + CreateMode.PERSISTENT); + } else if (op instanceof DeleteNodeFailSilent) { + DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op; + return Op.delete(dnfs.getPath(), -1); + } else if (op instanceof SetData) { + SetData sd = (SetData)op; + return Op.setData(sd.getPath(), sd.getData(), -1); + } else { + throw new UnsupportedOperationException("Unexpected ZKUtilOp type: " + + op.getClass().getName()); + } + } + + /** + * Use ZooKeeper's multi-update functionality. + * + * If all of the following are true: + * - runSequentialOnMultiFailure is true + * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*) + * Then: + * - we retry the operations one-by-one (sequentially) + * + * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi, + * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who + * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught). + * This will cause all operations in the multi to fail, however, because + * the NodeExistsException that zk.create throws will fail the multi transaction. + * In this case, if the previous conditions hold, the commands are run sequentially, which should + * result in the correct final state, but means that the operations will not run atomically. + * + * @throws KeeperException + */ + public static void multiOrSequential(ZooKeeperWatcher zkw, List ops, + boolean runSequentialOnMultiFailure) throws KeeperException { + if (zkw.getConfiguration().get("hbase.zookeeper.useMulti") != null) { + LOG.warn("hbase.zookeeper.useMulti is deprecated. Default to true always."); + } + if (ops == null) return; + + List zkOps = new LinkedList<>(); + for (ZKUtilOp op : ops) { + zkOps.add(toZooKeeperOp(zkw, op)); + } + try { + zkw.getRecoverableZooKeeper().multi(zkOps); + } catch (KeeperException ke) { + switch (ke.code()) { + case NODEEXISTS: + case NONODE: + case BADVERSION: + case NOAUTH: + // if we get an exception that could be solved by running sequentially + // (and the client asked us to), then break out and run sequentially + if (runSequentialOnMultiFailure) { + LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "." + + " Attempting to run operations sequentially because" + + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + "."); + processSequentially(zkw, ops); + break; + } + default: + throw ke; + } + } catch (InterruptedException ie) { + zkw.interruptedException(ie); + } + } + + private static void processSequentially(ZooKeeperWatcher zkw, List ops) + throws KeeperException, NoNodeException { + for (ZKUtilOp op : ops) { + if (op instanceof CreateAndFailSilent) { + createAndFailSilent(zkw, (CreateAndFailSilent) op); + } else if (op instanceof DeleteNodeFailSilent) { + deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op); + } else if (op instanceof SetData) { + setData(zkw, (SetData) op); + } else { + throw new UnsupportedOperationException("Unexpected ZKUtilOp type: " + + op.getClass().getName()); + } + } + } + + + // + // ZooKeeper cluster information + // + + static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode, StringBuilder sb) throws KeeperException { + sb.append("\n").append(hfileRefsZnode).append(": "); + for (String peerIdZnode : listChildrenNoWatch(zkw, hfileRefsZnode)) { + String znodeToProcess = joinZNode(hfileRefsZnode, peerIdZnode); + sb.append("\n").append(znodeToProcess).append(": "); + List peerHFileRefsZnodes = listChildrenNoWatch(zkw, znodeToProcess); + int size = peerHFileRefsZnodes.size(); + for (int i = 0; i < size; i++) { + sb.append(peerHFileRefsZnodes.get(i)); + if (i != size - 1) { + sb.append(", "); + } + } + } + } + + /** + * Gets the statistics from the given server. + * + * @param server The server to get the statistics from. + * @param timeout The socket timeout to use. + * @return The array of response strings. + * @throws IOException When the socket communication fails. + */ + public static String[] getServerStats(String server, int timeout) + throws IOException { + String[] sp = server.split(":"); + if (sp == null || sp.length == 0) { + return null; + } + + String host = sp[0]; + int port = sp.length > 1 ? Integer.parseInt(sp[1]) + : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT; + + InetSocketAddress sockAddr = new InetSocketAddress(host, port); + try (Socket socket = new Socket()) { + socket.connect(sockAddr, timeout); + + socket.setSoTimeout(timeout); + try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true); + BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { + out.println("stat"); + out.flush(); + ArrayList res = new ArrayList<>(); + while (true) { + String line = in.readLine(); + if (line != null) { + res.add(line); + } else { + break; + } + } + return res.toArray(new String[res.size()]); + } + } + } + + private static void logRetrievedMsg(final ZooKeeperWatcher zkw, + final String znode, final byte [] data, final boolean watcherSet) { + if (!LOG.isTraceEnabled()) return; + LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) + + " byte(s) of data from znode " + znode + + (watcherSet? " and set watcher; ": "; data=") + + (data == null? "null": data.length == 0? "empty": ( + znode.startsWith(zkw.znodePaths.metaZNodePrefix)? + getServerNameOrEmptyString(data): + znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)? + getServerNameOrEmptyString(data): + StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); + } + + private static String getServerNameOrEmptyString(final byte [] data) { + try { + return parseServerNameFrom(data).toString(); + } catch (DeserializationException e) { + return ""; + } + } + + /** + * Waits for HBase installation's base (parent) znode to become available. + * @throws IOException on ZK errors + */ + public static void waitForBaseZNode(Configuration conf) throws IOException { + LOG.info("Waiting until the base znode is available"); + String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); + ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf), + conf.getInt(HConstants.ZK_SESSION_TIMEOUT, + HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance); + + final int maxTimeMs = 10000; + final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS; + + KeeperException keeperEx = null; + try { + try { + for (int attempt = 0; attempt < maxNumAttempts; ++attempt) { + try { + if (zk.exists(parentZNode, false) != null) { + LOG.info("Parent znode exists: " + parentZNode); + keeperEx = null; + break; + } + } catch (KeeperException e) { + keeperEx = e; + } + Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS); + } + } finally { + zk.close(); + } + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + + if (keeperEx != null) { + throw new IOException(keeperEx); + } + } + + /** + * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}. + * Used when can't let a {@link DeserializationException} out w/o changing public API. + * @param e Exception to convert + * @return Converted exception + */ + public static KeeperException convert(final DeserializationException e) { + KeeperException ke = new KeeperException.DataInconsistencyException(); + ke.initCause(e); + return ke; + } + + /** + * Recursively print the current state of ZK (non-transactional) + * @param root name of the root directory in zk to print + */ + public static void logZKTree(ZooKeeperWatcher zkw, String root) { + if (!LOG.isDebugEnabled()) return; + LOG.debug("Current zk system:"); + String prefix = "|-"; + LOG.debug(prefix + root); + try { + logZKTree(zkw, root, prefix); + } catch (KeeperException e) { + throw new RuntimeException(e); + } + } + + /** + * Helper method to print the current state of the ZK tree. + * @see #logZKTree(ZooKeeperWatcher, String) + * @throws KeeperException if an unexpected exception occurs + */ + protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) + throws KeeperException { + List children = listChildrenNoWatch(zkw, root); + if (children == null) return; + for (String child : children) { + LOG.debug(prefix + child); + String node = joinZNode(root.equals("/") ? "" : root, child); + logZKTree(zkw, node, prefix + "---"); + } + } + + + /** + * NOTE: This is copy of ProtobufUtil#parseServerNameFrom. + * This one function was preventing move of many functions from hbase-client#ZKUtil to this + * class which in turn was blocking larger hbase-zookeeper dependency untangle. + * + * Get a ServerName from the passed in data bytes. + * @param data Data with a serialize server name in it; can handle the old style + * servername where servername was host and port. Works too with data that + * begins w/ the pb 'PBUF' magic and that is then followed by a protobuf that + * has a serialized {@link ServerName} in it. + * @return Returns null if data is null else converts passed data + * to a ServerName instance. + * @throws DeserializationException + */ + static ServerName parseServerNameFrom(final byte [] data) throws DeserializationException { + if (data == null || data.length <= 0) return null; + if (ProtobufMagic.isPBMagicPrefix(data)) { + int prefixLen = ProtobufMagic.lengthOfPBMagic(); + try { + ZooKeeperProtos.Master rss = + ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen); + org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn = + rss.getMaster(); + return ServerName.valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode()); + } catch (/*InvalidProtocolBufferException*/IOException e) { + // A failed parse of the znode is pretty catastrophic. Rather than loop + // retrying hoping the bad bytes will changes, and rather than change + // the signature on this method to add an IOE which will send ripples all + // over the code base, throw a RuntimeException. This should "never" happen. + // Fail fast if it does. + throw new DeserializationException(e); + } + } + // The str returned could be old style -- pre hbase-1502 -- which was + // hostname and port seperated by a colon rather than hostname, port and + // startcode delimited by a ','. + String str = Bytes.toString(data); + int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR); + if (index != -1) { + // Presume its ServerName serialized with versioned bytes. + return ServerName.parseVersionedServerName(data); + } + // Presume it a hostname:port format. + String hostname = Addressing.parseHostname(str); + int port = Addressing.parsePort(str); + return ServerName.valueOf(hostname, port, -1L); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java similarity index 94% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index 30913dc2eb..fc676180f4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-zookeeper/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,9 +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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.security.UserGroupInformation; @@ -85,7 +80,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { // listeners to be notified private final List listeners = new CopyOnWriteArrayList<>(); - // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL + // Used by ZooKeeperUtil:waitForZKConnectionIfAuthenticating to wait for SASL // negotiation to complete public CountDownLatch saslLatch = new CountDownLatch(1); @@ -131,7 +126,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { this.abortable = abortable; this.znodePaths = new ZNodePaths(conf); PendingWatcher pendingWatcher = new PendingWatcher(); - this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier); + this.recoverableZooKeeper = ZooKeeperUtil.connect(conf, quorum, pendingWatcher, identifier); pendingWatcher.prepare(this); if (canCreateBaseZNode) { try { @@ -151,15 +146,15 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { private void createBaseZNodes() throws ZooKeeperConnectionException { try { // Create all the necessary "directories" of znodes - 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); + ZooKeeperUtil.createWithParents(this, znodePaths.baseZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.rsZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.drainingZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.tableZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.splitLogZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.tableLockZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.recoveringRegionsZNode); + ZooKeeperUtil.createAndFailSilent(this, znodePaths.masterMaintZNode); } catch (KeeperException e) { throw new ZooKeeperConnectionException( prefix("Unexpected KeeperException creating base node"), e); @@ -190,7 +185,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * perms. */ public void checkAndSetZNodeAcls() { - if (!ZKUtil.isSecureZooKeeper(getConfiguration())) { + if (!ZooKeeperUtil.isSecureZooKeeper(getConfiguration())) { LOG.info("not a secure deployment, proceeding"); return; } @@ -222,9 +217,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { List children = recoverableZooKeeper.getChildren(znode, false); for (String child : children) { - setZnodeAclsRecursive(ZKUtil.joinZNode(znode, child)); + setZnodeAclsRecursive(ZooKeeperUtil.joinZNode(znode, child)); } - List acls = ZKUtil.createACL(this, znode, true); + List acls = ZooKeeperUtil.createACL(this, znode, true); LOG.info("Setting ACLs for znode:" + znode + " , acl:" + acls); recoverableZooKeeper.setAcl(znode, acls, -1); } @@ -372,7 +367,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @throws KeeperException */ public List getMetaReplicaNodes() throws KeeperException { - List childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode); + List childrenOfBaseNode = ZooKeeperUtil.listChildrenNoWatch(this, znodePaths.baseZNode); List metaReplicaNodes = new ArrayList<>(2); if (childrenOfBaseNode != null) { String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java similarity index 100% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java similarity index 100% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestMetricsZooKeeper.java diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java similarity index 93% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java index 53dcdbc50c..0fc1ee5bb5 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java @@ -23,7 +23,6 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.security.UserGroupInformation; @@ -49,7 +48,7 @@ public class TestZKUtil { conf.set(Superusers.SUPERUSER_CONF_KEY, "user1"); String node = "/hbase/testUnsecure"; ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); - List aclList = ZKUtil.createACL(watcher, node, false); + List aclList = ZooKeeperUtil.createACL(watcher, node, false); Assert.assertEquals(aclList.size(), 1); Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next())); } @@ -60,7 +59,7 @@ public class TestZKUtil { conf.set(Superusers.SUPERUSER_CONF_KEY, "user1"); String node = "/hbase/testSecuritySingleSuperuser"; ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); - List aclList = ZKUtil.createACL(watcher, node, true); + List aclList = ZooKeeperUtil.createACL(watcher, node, true); Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1")))); Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next())); @@ -72,7 +71,7 @@ public class TestZKUtil { conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3"); String node = "/hbase/testCreateACL"; ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); - List aclList = ZKUtil.createACL(watcher, node, true); + List aclList = ZooKeeperUtil.createACL(watcher, node, true); Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1")))); Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2")))); @@ -88,7 +87,7 @@ public class TestZKUtil { UserGroupInformation.setLoginUser(UserGroupInformation.createRemoteUser("user4")); String node = "/hbase/testCreateACL"; ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); - List aclList = ZKUtil.createACL(watcher, node, true); + List aclList = ZooKeeperUtil.createACL(watcher, node, true); Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1")))); Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", "")))); @@ -108,7 +107,7 @@ public class TestZKUtil { Mockito.doThrow(new InterruptedException()).when(recoverableZk) .getChildren(zkw.znodePaths.baseZNode, null); try { - ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode); + ZooKeeperUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode); } catch (KeeperException.SystemErrorException e) { // expected return; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java similarity index 94% rename from hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java rename to hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java index de2ec2a236..750dde25d9 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -42,7 +41,7 @@ public class TestZooKeeperWatcher { assertTrue(watcher.isClientReadable(watcher.znodePaths.masterAddressZNode)); assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode)); assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode)); - assertTrue(watcher.isClientReadable(ZKUtil.joinZNode(watcher.znodePaths.tableZNode, "foo"))); + assertTrue(watcher.isClientReadable(ZooKeeperUtil.joinZNode(watcher.znodePaths.tableZNode, "foo"))); assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode)); assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode)); diff --git a/pom.xml b/pom.xml index 31abcc98d7..8709561a2e 100755 --- a/pom.xml +++ b/pom.xml @@ -68,6 +68,7 @@ hbase-resource-bundle hbase-http hbase-server + hbase-zookeeper hbase-thrift hbase-shell hbase-protocol-shaded @@ -1649,6 +1650,18 @@ test-jar test
+ + hbase-zookeeper + org.apache.hbase + ${project.version} + + + hbase-zookeeper + org.apache.hbase + ${project.version} + test-jar + test + hbase-server org.apache.hbase -- 2.14.1