View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client.replication;
21  
22  import java.io.IOException;
23  
24  import org.apache.commons.lang.NotImplementedException;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.HConstants;
27  import org.apache.hadoop.hbase.client.HConnection;
28  import org.apache.hadoop.hbase.client.HConnectionManager;
29  import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
30  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31  import org.apache.zookeeper.KeeperException;
32  
33  /**
34   * <p>
35   * This class provides the administrative interface to HBase cluster
36   * replication. In order to use it, the cluster and the client using
37   * ReplicationAdmin must be configured with <code>hbase.replication</code>
38   * set to true.
39   * </p>
40   * <p>
41   * Adding a new peer results in creating new outbound connections from every
42   * region server to a subset of region servers on the slave cluster. Each
43   * new stream of replication will start replicating from the beginning of the
44   * current HLog, meaning that edits from that past will be replicated.
45   * </p>
46   * <p>
47   * Removing a peer is a destructive and irreversible operation that stops
48   * all the replication streams for the given cluster and deletes the metadata
49   * used to keep track of the replication state.
50   * </p>
51   * <p>
52   * Enabling and disabling peers is currently not supported.
53   * </p>
54   * <p>
55   * As cluster replication is still experimental, a kill switch is provided
56   * in order to stop all replication-related operations, see
57   * {@link #setReplicating(boolean)}. When setting it back to true, the new
58   * state of all the replication streams will be unknown and may have holes.
59   * Use at your own risk.
60   * </p>
61   * <p>
62   * To see which commands are available in the shell, type
63   * <code>replication</code>.
64   * </p>
65   */
66  public class ReplicationAdmin {
67  
68    private final ReplicationZookeeper replicationZk;
69    private final HConnection connection;
70  
71    /**
72     * Constructor that creates a connection to the local ZooKeeper ensemble.
73     * @param conf Configuration to use
74     * @throws IOException if the connection to ZK cannot be made
75     * @throws RuntimeException if replication isn't enabled.
76     */
77    public ReplicationAdmin(Configuration conf) throws IOException {
78      if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
79        throw new RuntimeException("hbase.replication isn't true, please " +
80            "enable it in order to use replication");
81      }
82      this.connection = HConnectionManager.getConnection(conf);
83      ZooKeeperWatcher zkw = this.connection.getZooKeeperWatcher();
84      try {
85        this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
86      } catch (KeeperException e) {
87        throw new IOException("Unable setup the ZooKeeper connection", e);
88      }
89    }
90  
91    /**
92     * Add a new peer cluster to replicate to.
93     * @param id a short that identifies the cluster
94     * @param clusterKey the concatenation of the slave cluster's
95     * <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>
96     * @throws IllegalStateException if there's already one slave since
97     * multi-slave isn't supported yet.
98     */
99    public void addPeer(String id, String clusterKey) throws IOException {
100     this.replicationZk.addPeer(id, clusterKey);
101   }
102 
103   /**
104    * Removes a peer cluster and stops the replication to it.
105    * @param id a short that identifies the cluster
106    */
107   public void removePeer(String id) throws IOException {
108     this.replicationZk.removePeer(id);
109   }
110 
111   /**
112    * Restart the replication stream to the specified peer.
113    * @param id a short that identifies the cluster
114    */
115   public void enablePeer(String id) {
116     throw new NotImplementedException("Not implemented");
117   }
118 
119   /**
120    * Stop the replication stream to the specified peer.
121    * @param id a short that identifies the cluster
122    */
123   public void disablePeer(String id) {
124     throw new NotImplementedException("Not implemented");
125   }
126 
127   /**
128    * Get the number of slave clusters the local cluster has.
129    * @return number of slave clusters
130    */
131   public int getPeersCount() {
132     return this.replicationZk.listPeersIdsAndWatch().size();
133   }
134 
135   /**
136    * Get the current status of the kill switch, if the cluster is replicating
137    * or not.
138    * @return true if the cluster is replicated, otherwise false
139    */
140   public boolean getReplicating() throws IOException {
141     try {
142       return this.replicationZk.getReplication();
143     } catch (KeeperException e) {
144       throw new IOException("Couldn't get the replication status");
145     }
146   }
147 
148   /**
149    * Kill switch for all replication-related features
150    * @param newState true to start replication, false to stop it.
151    * completely
152    * @return the previous state
153    */
154   public boolean setReplicating(boolean newState) throws IOException {
155     boolean prev = true;
156     try {
157       prev = getReplicating();
158       this.replicationZk.setReplicating(newState);
159     } catch (KeeperException e) {
160       throw new IOException("Unable to set the replication state", e);
161     }
162     return prev;
163   }
164 
165   /**
166    * Get the ZK-support tool created and used by this object for replication.
167    * @return the ZK-support tool
168    */
169   ReplicationZookeeper getReplicationZk() {
170     return replicationZk;
171   }
172 }