Skip to content

Commit

Permalink
HBASE-26614 Refactor code related to "dump"ing ZK nodes (apache#3969)
Browse files Browse the repository at this point in the history
The code starting at `ZKUtil.dump(ZKWatcher)` is a small mess – it has cyclic dependencies woven
through itself, `ZKWatcher` and `RecoverableZooKeeper`. It also initializes a static variable in
`ZKUtil` through the factory for `RecoverableZooKeeper` instances. Let's decouple and clean it
up.

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Josh Elser <elserj@apache.org>
  • Loading branch information
ndimiduk committed Jan 24, 2022
1 parent 5aa0fd2 commit 0da06b0
Show file tree
Hide file tree
Showing 11 changed files with 381 additions and 331 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.hadoop.util.ToolRunner;
Expand Down Expand Up @@ -142,7 +141,7 @@ protected int doWork() throws Exception {
private void testZNodeACLs() throws IOException, KeeperException, InterruptedException {

ZKWatcher watcher = new ZKWatcher(conf, "IntegrationTestZnodeACLs", null);
RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher);
RecoverableZooKeeper zk = RecoverableZooKeeper.connect(this.conf, watcher);

String baseZNode = watcher.getZNodePaths().baseZNode;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKDump;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.util.Tool;
Expand Down Expand Up @@ -241,7 +242,7 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
} else {
// use ZK instead
System.out.print("Dumping replication znodes via ZooKeeper:");
System.out.println(ZKUtil.getReplicationZnodesDump(zkw));
System.out.println(ZKDump.getReplicationZnodesDump(zkw));
}
return (0);
} catch (IOException e) {
Expand Down
6 changes: 3 additions & 3 deletions hbase-server/src/main/resources/hbase-webapps/master/zk.jsp
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@
--%>
<%@ page contentType="text/html;charset=UTF-8"
import="org.apache.commons.lang3.StringEscapeUtils"
import="org.apache.hadoop.hbase.zookeeper.ZKUtil"
import="org.apache.hadoop.hbase.zookeeper.ZKWatcher"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.zookeeper.ZKDump"
import="org.apache.hadoop.hbase.zookeeper.ZKWatcher"
%>
<%
HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
Expand All @@ -38,7 +38,7 @@
</div>
<div class="row">
<div class="span12">
<pre><%= StringEscapeUtils.escapeHtml4(ZKUtil.dump(watcher).trim()) %></pre>
<pre><%= StringEscapeUtils.escapeHtml4(ZKDump.dump(watcher).trim()) %></pre>
</div>
</div>
</div>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;

import org.junit.After;
Expand Down Expand Up @@ -282,7 +281,7 @@ public void testBulkLoadReplicationActiveActive() throws Exception {
// Verify hfile-refs for 1:ns_peer1, expect is empty
MiniZooKeeperCluster zkCluster = UTIL1.getZkCluster();
ZKWatcher watcher = new ZKWatcher(UTIL1.getConfiguration(), "TestZnodeHFiles-refs", null);
RecoverableZooKeeper zk = ZKUtil.connect(UTIL1.getConfiguration(), watcher);
RecoverableZooKeeper zk = RecoverableZooKeeper.connect(UTIL1.getConfiguration(), watcher);
ZKReplicationQueueStorage replicationQueueStorage =
new ZKReplicationQueueStorage(watcher, UTIL1.getConfiguration());
Set<String> hfiles = replicationQueueStorage.getAllHFileRefs();
Expand Down
2 changes: 1 addition & 1 deletion hbase-shell/src/main/ruby/hbase/admin.rb
Original file line number Diff line number Diff line change
Expand Up @@ -452,7 +452,7 @@ def zk_dump
)
zk = @zk_wrapper.getRecoverableZooKeeper.getZooKeeper
@zk_main = org.apache.zookeeper.ZooKeeperMain.new(zk)
org.apache.hadoop.hbase.zookeeper.ZKUtil.dump(@zk_wrapper)
org.apache.hadoop.hbase.zookeeper.ZKDump.dump(@zk_wrapper)
end

#----------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private MetaTableLocator() {
* @return server name or null if we failed to get the data.
*/
@RestrictedApi(explanation = "Should only be called in tests or ZKUtil", link = "",
allowedOnPath = ".*/src/test/.*|.*/ZKUtil\\.java")
allowedOnPath = ".*/src/test/.*|.*/ZKDump\\.java")
public static ServerName getMetaRegionLocation(final ZKWatcher zkw) {
try {
RegionState state = getMetaRegionState(zkw);
Expand All @@ -75,7 +75,7 @@ public static ServerName getMetaRegionLocation(final ZKWatcher zkw) {
* @return server name
*/
@RestrictedApi(explanation = "Should only be called in self or ZKUtil", link = "",
allowedOnPath = ".*(MetaTableLocator|ZKUtil)\\.java")
allowedOnPath = ".*(MetaTableLocator|ZKDump)\\.java")
public static ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
try {
RegionState state = getMetaRegionState(zkw, replicaId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
Expand Down Expand Up @@ -84,6 +86,57 @@ public class RecoverableZooKeeper {
private final String quorumServers;
private final int maxMultiSize;

/**
* See {@link #connect(Configuration, String, Watcher, String)}
*/
public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
throws IOException {
String ensemble = ZKConfig.getZKQuorumServersString(conf);
return connect(conf, ensemble, watcher);
}

/**
* See {@link #connect(Configuration, String, Watcher, String)}
*/
public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
Watcher watcher)
throws IOException {
return connect(conf, ensemble, watcher, null);
}

/**
* 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
* @param ensemble ZooKeeper servers quorum string
* @param identifier value used to identify this client instance.
* @return connection to zookeeper
* @throws IOException if unable to connect to zk or config problem
*/
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("{} opening connection to ZooKeeper ensemble={}", identifier, 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);
int multiMaxSize = conf.getInt("zookeeper.multi.max.size", 1024*1024);
return new RecoverableZooKeeper(ensemble, timeout, watcher,
retry, retryIntervalMillis, maxSleepTime, identifier, multiMaxSize);
}

@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
justification="None. Its always been this way.")
public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
Expand Down
Loading

0 comments on commit 0da06b0

Please sign in to comment.