Skip to content

Commit

Permalink
HBASE-28554 TestZooKeeperScanPolicyObserver and TestAdminShell fail 1…
Browse files Browse the repository at this point in the history
…00% of times on flaky dashboard (#5859)

Signed-off-by: Bryan Beaudreault <[email protected]>
(cherry picked from commit 4230c42)
  • Loading branch information
Apache9 committed Apr 28, 2024
1 parent ed50d20 commit d6beb61
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
Expand Down Expand Up @@ -83,7 +84,12 @@ public static void tearDown() throws Exception {

private void setExpireBefore(long time)
throws KeeperException, InterruptedException, IOException {
ZooKeeper zk = UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().getZooKeeper();
RecoverableZooKeeper recoverableZk = UTIL.getZooKeeperWatcher().getRecoverableZooKeeper();
// we need to call this for setting up the zookeeper connection
recoverableZk.reconnectAfterExpiration();
// we have to use the original ZooKeeper as the RecoverableZooKeeper will append a magic prefix
// for the data stored on zookeeper
ZooKeeper zk = recoverableZk.getZooKeeper();
if (zk.exists(ZooKeeperScanPolicyObserver.NODE, false) == null) {
zk.create(ZooKeeperScanPolicyObserver.NODE, Bytes.toBytes(time), ZooDefs.Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
Expand Down
2 changes: 0 additions & 2 deletions hbase-shell/src/main/ruby/hbase/admin.rb
Original file line number Diff line number Diff line change
Expand Up @@ -460,8 +460,6 @@ def zk_dump
'admin',
nil
)
zk = @zk_wrapper.getRecoverableZooKeeper.getZooKeeper
@zk_main = org.apache.zookeeper.ZooKeeperMain.new(zk)
org.apache.hadoop.hbase.zookeeper.ZKDump.dump(@zk_wrapper)
end

Expand Down

0 comments on commit d6beb61

Please sign in to comment.