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.client.replication;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.client.HConnection;
27 import org.apache.hadoop.hbase.client.HConnectionManager;
28 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
29 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
30 import org.apache.zookeeper.KeeperException;
31
32 import java.io.Closeable;
33 import java.io.IOException;
34 import java.util.Map;
35
36 /**
37 * <p>
38 * This class provides the administrative interface to HBase cluster
39 * replication. In order to use it, the cluster and the client using
40 * ReplicationAdmin must be configured with <code>hbase.replication</code>
41 * set to true.
42 * </p>
43 * <p>
44 * Adding a new peer results in creating new outbound connections from every
45 * region server to a subset of region servers on the slave cluster. Each
46 * new stream of replication will start replicating from the beginning of the
47 * current HLog, meaning that edits from that past will be replicated.
48 * </p>
49 * <p>
50 * Removing a peer is a destructive and irreversible operation that stops
51 * all the replication streams for the given cluster and deletes the metadata
52 * used to keep track of the replication state.
53 * </p>
54 * <p>
55 * Enabling and disabling peers is currently not supported.
56 * </p>
57 * <p>
58 * As cluster replication is still experimental, a kill switch is provided
59 * in order to stop all replication-related operations, see
60 * {@link #setReplicating(boolean)}. When setting it back to true, the new
61 * state of all the replication streams will be unknown and may have holes.
62 * Use at your own risk.
63 * </p>
64 * <p>
65 * To see which commands are available in the shell, type
66 * <code>replication</code>.
67 * </p>
68 */
69 public class ReplicationAdmin implements Closeable {
70 private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
71
72 private final ReplicationZookeeper replicationZk;
73 private final HConnection connection;
74
75 /**
76 * Constructor that creates a connection to the local ZooKeeper ensemble.
77 * @param conf Configuration to use
78 * @throws IOException if the connection to ZK cannot be made
79 * @throws RuntimeException if replication isn't enabled.
80 */
81 public ReplicationAdmin(Configuration conf) throws IOException {
82 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
83 throw new RuntimeException("hbase.replication isn't true, please " +
84 "enable it in order to use replication");
85 }
86 this.connection = HConnectionManager.getConnection(conf);
87 ZooKeeperWatcher zkw = createZooKeeperWatcher();
88 try {
89 this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
90 } catch (KeeperException e) {
91 throw new IOException("Unable setup the ZooKeeper connection", e);
92 }
93 }
94
95 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
96 return new ZooKeeperWatcher(connection.getConfiguration(),
97 "Replication Admin", new Abortable() {
98 @Override
99 public void abort(String why, Throwable e) {
100 LOG.error(why, e);
101 System.exit(1);
102 }
103
104 @Override
105 public boolean isAborted() {
106 return false;
107 }
108
109 });
110 }
111
112
113 /**
114 * Add a new peer cluster to replicate to.
115 * @param id a short that identifies the cluster
116 * @param clusterKey the concatenation of the slave cluster's
117 * <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>
118 * @throws IllegalStateException if there's already one slave since
119 * multi-slave isn't supported yet.
120 */
121 public void addPeer(String id, String clusterKey) throws IOException {
122 this.replicationZk.addPeer(id, clusterKey);
123 }
124
125 /**
126 * Removes a peer cluster and stops the replication to it.
127 * @param id a short that identifies the cluster
128 */
129 public void removePeer(String id) throws IOException {
130 this.replicationZk.removePeer(id);
131 }
132
133 /**
134 * Restart the replication stream to the specified peer.
135 * @param id a short that identifies the cluster
136 */
137 public void enablePeer(String id) throws IOException {
138 this.replicationZk.enablePeer(id);
139 }
140
141 /**
142 * Stop the replication stream to the specified peer.
143 * @param id a short that identifies the cluster
144 */
145 public void disablePeer(String id) throws IOException {
146 this.replicationZk.disablePeer(id);
147 }
148
149 /**
150 * Get the number of slave clusters the local cluster has.
151 * @return number of slave clusters
152 */
153 public int getPeersCount() {
154 return this.replicationZk.listPeersIdsAndWatch().size();
155 }
156
157 /**
158 * Map of this cluster's peers for display.
159 * @return A map of peer ids to peer cluster keys
160 */
161 public Map<String, String> listPeers() {
162 return this.replicationZk.listPeers();
163 }
164
165 /**
166 * Get the current status of the kill switch, if the cluster is replicating
167 * or not.
168 * @return true if the cluster is replicated, otherwise false
169 */
170 public boolean getReplicating() throws IOException {
171 try {
172 return this.replicationZk.getReplication();
173 } catch (KeeperException e) {
174 throw new IOException("Couldn't get the replication status");
175 }
176 }
177
178 /**
179 * Kill switch for all replication-related features
180 * @param newState true to start replication, false to stop it.
181 * completely
182 * @return the previous state
183 */
184 public boolean setReplicating(boolean newState) throws IOException {
185 boolean prev = true;
186 try {
187 prev = getReplicating();
188 this.replicationZk.setReplication(newState);
189 } catch (KeeperException e) {
190 throw new IOException("Unable to set the replication state", e);
191 }
192 return prev;
193 }
194
195 /**
196 * Get the ZK-support tool created and used by this object for replication.
197 * @return the ZK-support tool
198 */
199 ReplicationZookeeper getReplicationZk() {
200 return replicationZk;
201 }
202
203 @Override
204 public void close() throws IOException {
205 if (this.connection != null) {
206 this.connection.close();
207 }
208 }
209 }