View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.replication;
20  
21  import java.util.List;
22  
23  import org.apache.hadoop.conf.Configuration;
24  import org.apache.hadoop.hbase.Abortable;
25  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
26  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
27  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
28  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
29  import org.apache.zookeeper.KeeperException;
30  
31  
32  /**
33   * This is a base class for maintaining replication state in zookeeper.
34   */
35  public abstract class ReplicationStateZKBase {
36  
37    /**
38     * The name of the znode that contains the replication status of a remote slave (i.e. peer)
39     * cluster.
40     */
41    protected final String peerStateNodeName;
42    /** The name of the base znode that contains all replication state. */
43    protected final String replicationZNode;
44    /** The name of the znode that contains a list of all remote slave (i.e. peer) clusters. */
45    protected final String peersZNode;
46    /** The name of the znode that contains all replication queues */
47    protected final String queuesZNode;
48    /** The cluster key of the local cluster */
49    protected final String ourClusterKey;
50    protected final ZooKeeperWatcher zookeeper;
51    protected final Configuration conf;
52    protected final Abortable abortable;
53  
54    // Public for testing
55    public static final byte[] ENABLED_ZNODE_BYTES =
56        toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
57    public static final byte[] DISABLED_ZNODE_BYTES =
58        toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
59  
60    public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
61        Abortable abortable) {
62      this.zookeeper = zookeeper;
63      this.conf = conf;
64      this.abortable = abortable;
65  
66      String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
67      String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
68      String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
69      this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
70      this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
71      this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
72      this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
73      this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
74    }
75  
76    public List<String> getListOfReplicators() {
77      List<String> result = null;
78      try {
79        result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
80      } catch (KeeperException e) {
81        this.abortable.abort("Failed to get list of replicators", e);
82      }
83      return result;
84    }
85  
86    /**
87     * @param state
88     * @return Serialized protobuf of <code>state</code> with pb magic prefix prepended suitable for
89     *         use as content of a peer-state znode under a peer cluster id as in
90     *         /hbase/replication/peers/PEER_ID/peer-state.
91     */
92    protected static byte[] toByteArray(final ZooKeeperProtos.ReplicationState.State state) {
93      byte[] bytes =
94          ZooKeeperProtos.ReplicationState.newBuilder().setState(state).build().toByteArray();
95      return ProtobufUtil.prependPBMagic(bytes);
96    }
97  
98    protected boolean peerExists(String id) throws KeeperException {
99      return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
100   }
101 
102   /**
103    * Determine if a ZK path points to a peer node.
104    * @param path path to be checked
105    * @return true if the path points to a peer node, otherwise false
106    */
107   protected boolean isPeerPath(String path) {
108     return path.split("/").length == peersZNode.split("/").length + 1;
109   }
110 }