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