hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HBASE-546) Use Zookeeper in HBase
Date Wed, 21 Jan 2009 22:47:59 GMT

    [ https://issues.apache.org/jira/browse/HBASE-546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12665994#action_12665994

stack commented on HBASE-546:

For zookeeper.servers, you specify a quorum by listing the quorum members?

When single-instance of zk that has been started by hbase runs, where does it write its logs.
 Can that be configurable?

What if I specify a full-path for zookeeper.znode.rootserver, will that write root region
location outside of zookeeper.znode.parent?

(Regards a question you asked on IRC a few days ago) If things like DEFAULT_ZOOKEEPER_SERVERS
, are only used in one place, I'd say don't need to be in HConstants... just do it in place
used.  If used more than once, its good to define HConstants AND they are used by more than
one class (otherwise, do the define inside that class)

Should below be synchronized nitay?

+    private ZooKeeperWrapper getZooKeeperWrapper() throws IOException {

Is there danger that two threads could be asking for it at about same time?

This is an interesting change Nitay: -  public HServerAddress findRootRegion();... removing
it from the Master.  I like it.  ZK rules now!

+ * Copyright 2008 The Apache Software Foundation
... should be 2009

Below has trailing '\'

+ * Wraps a ZooKeeper instance and adds HBase specific functionality.\

Can these be final in ZKWrapper?

+  private ZooKeeper zooKeeper;
+  private WatcherWrapper watcher;

... same in HRS:

+  private ZooKeeperWrapper zooKeeperWrapper;

In below...

+    rootRegionZNode = parentZNode + "/" + rootServerZNodeName;

... does ZK have a define for path separator?

We don't throw exception if we fail to get root:

+    try {
+      data = zooKeeper.getData(rootRegionZNode, false, null);
+    } catch (InterruptedException e) {
+      return null;
+    } catch (KeeperException e) {
+      return null;
+    }

... is that good?  Does caller handle null?

Whats story w/ data in ZK?  Its byte arrays?  Should they be UTF-8?  If so, Bytes.toBytes
over in hbase util might help.  E.g. rather than +    String addressString = new String(data);
which could give different answers if the client was in different local than original writer,
be explicit its utf-8 and do Bytes.toBytes(....) when writing and Bytes.toString(...  when

> Use Zookeeper in HBase
> ----------------------
>                 Key: HBASE-546
>                 URL: https://issues.apache.org/jira/browse/HBASE-546
>             Project: Hadoop HBase
>          Issue Type: Improvement
>            Reporter: Bryan Duxbury
>            Assignee: Jean-Daniel Cryans
>            Priority: Critical
>             Fix For: 0.20.0
>         Attachments: DistributedLockInterface.java, hbase-546-scripts-v2.patch, hbase-546-scripts.patch,
HBASE-546-v2.patch, hbase-546-v3.patch, HBASE-546.patch, zookeeper-config.patch
> Zookeeper =~ Chubby. This means that we could take advantage of a distributed lock manager
to coordinate things like failover masters, regionservers staying online when master is dead,
atomic region->regionserver assignments, etc. There are a lot of opportunities for improvements
here. Please add discussions of particular features in comments or sub-tasks.

This message is automatically generated by JIRA.
You can reply to this email to add a comment to the issue online.

View raw message