This is an automated email from the ASF dual-hosted git repository.
stoty pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-3 by this push:
new e64667f6a31 HBASE-29313 RecoverableZooKeeper.getZooKeeper() returns
null for new object (#6987)
e64667f6a31 is described below
commit e64667f6a31a79657d36a7416dbd80d8475458fb
Author: Istvan Toth <[email protected]>
AuthorDate: Thu May 15 16:52:04 2025 +0200
HBASE-29313 RecoverableZooKeeper.getZooKeeper() returns null for new object
(#6987)
Signed-off-by: Duo Zhang <[email protected]>
(cherry picked from commit d9b1aa108960bafcb5edaa676833d85a6025c4a9)
---
.../hadoop/hbase/zookeeper/RecoverableZooKeeper.java | 14 +++++++++++++-
1 file changed, 13 insertions(+), 1 deletion(-)
diff --git
a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 8537dd12c5b..feebfa58a15 100644
---
a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++
b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -725,8 +725,20 @@ public class RecoverableZooKeeper {
return zk == null ? null : zk.getState();
}
+ /**
+ * Returns the wrapped ZooKeeper client. If the wrapped client hasn't been
created yet then tries
+ * create it first.
+ * @return the wrapped ZK client of null if the creation has failed.
+ */
public synchronized ZooKeeper getZooKeeper() {
- return zk;
+ // Callers expect an initialized ZooKeeper instance
+ // Pre HBASE-28529 the constructor used to call checkZk()
+ try {
+ return checkZk();
+ } catch (Exception x) {
+ LOG.warn("Failed to initialize Zookeeper object, returning null", x);
+ return null;
+ }
}
public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx)
throws KeeperException {