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.client.replication;
20  
21  import java.io.Closeable;
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.HashMap;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.Map.Entry;
29  import org.apache.commons.lang.StringUtils;
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.classification.InterfaceStability;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.Abortable;
36  import org.apache.hadoop.hbase.HColumnDescriptor;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HTableDescriptor;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.client.HConnection;
41  import org.apache.hadoop.hbase.client.HConnectionManager;
42  import org.apache.hadoop.hbase.replication.ReplicationException;
43  import org.apache.hadoop.hbase.replication.ReplicationFactory;
44  import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
45  import org.apache.hadoop.hbase.replication.ReplicationPeers;
46  import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
47  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
48  
49  import com.google.common.annotations.VisibleForTesting;
50  
51  /**
52   * <p>
53   * This class provides the administrative interface to HBase cluster
54   * replication. In order to use it, the cluster and the client using
55   * ReplicationAdmin must be configured with <code>hbase.replication</code>
56   * set to true.
57   * </p>
58   * <p>
59   * Adding a new peer results in creating new outbound connections from every
60   * region server to a subset of region servers on the slave cluster. Each
61   * new stream of replication will start replicating from the beginning of the
62   * current HLog, meaning that edits from that past will be replicated.
63   * </p>
64   * <p>
65   * Removing a peer is a destructive and irreversible operation that stops
66   * all the replication streams for the given cluster and deletes the metadata
67   * used to keep track of the replication state.
68   * </p>
69   * <p>
70   * To see which commands are available in the shell, type
71   * <code>replication</code>.
72   * </p>
73   */
74  @InterfaceAudience.Public
75  @InterfaceStability.Evolving
76  public class ReplicationAdmin implements Closeable {
77    private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
78  
79    public static final String TNAME = "tableName";
80    public static final String CFNAME = "columnFamlyName";
81  
82    // only Global for now, can add other type
83    // such as, 1) no global replication, or 2) the table is replicated to this cluster, etc.
84    public static final String REPLICATIONTYPE = "replicationType";
85    public static final String REPLICATIONGLOBAL = Integer
86        .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
87  
88    private final HConnection connection;
89    // TODO: replication should be managed by master. All the classes except ReplicationAdmin should
90    // be moved to hbase-server. Resolve it in HBASE-11392.
91    private final ReplicationQueuesClient replicationQueuesClient;
92    private final ReplicationPeers replicationPeers;
93    /**
94     * A watcher used by replicationPeers and replicationQueuesClient. Keep reference so can dispose
95     * on {@link #close()}.
96     */
97    private final ZooKeeperWatcher zkw;
98  
99    /**
100    * Constructor that creates a connection to the local ZooKeeper ensemble.
101    * @param conf Configuration to use
102    * @throws IOException if an internal replication error occurs
103    * @throws RuntimeException if replication isn't enabled.
104    */
105   public ReplicationAdmin(Configuration conf) throws IOException {
106     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
107         HConstants.REPLICATION_ENABLE_DEFAULT)) {
108       throw new RuntimeException("hbase.replication isn't true, please " +
109           "enable it in order to use replication");
110     }
111     this.connection = HConnectionManager.getConnection(conf);
112     zkw = createZooKeeperWatcher();
113     try {
114       this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
115       this.replicationPeers.init();
116       this.replicationQueuesClient =
117           ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
118       this.replicationQueuesClient.init();
119 
120     } catch (ReplicationException e) {
121       throw new IOException("Error initializing the replication admin client.", e);
122     }
123   }
124 
125   private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
126     // This Abortable doesn't 'abort'... it just logs.
127     return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
128       @Override
129       public void abort(String why, Throwable e) {
130         LOG.error(why, e);
131         // We used to call system.exit here but this script can be embedded by other programs that
132         // want to do replication stuff... so inappropriate calling System.exit. Just log for now.
133       }
134 
135       @Override
136       public boolean isAborted() {
137         return false;
138       }
139     });
140   }
141 
142   /**
143    * Add a new peer cluster to replicate to.
144    * @param id a short name that identifies the cluster
145    * @param clusterKey the concatenation of the slave cluster's
146    * <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>
147    * @throws IllegalStateException if there's already one slave since
148    * multi-slave isn't supported yet.
149    * @deprecated Use addPeer(String, ReplicationPeerConfig, Map) instead.
150    */
151   @Deprecated
152   public void addPeer(String id, String clusterKey) throws ReplicationException {
153     this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
154   }
155 
156   @Deprecated
157   public void addPeer(String id, String clusterKey, String tableCFs)
158     throws ReplicationException {
159     this.replicationPeers.addPeer(id,
160       new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
161   }
162 
163   /**
164    * Add a new remote slave cluster for replication.
165    * @param id a short name that identifies the cluster
166    * @param peerConfig configuration for the replication slave cluster
167    * @param tableCfs the table and column-family list which will be replicated for this peer.
168    * A map from tableName to column family names. An empty collection can be passed
169    * to indicate replicating all column families. Pass null for replicating all table and column
170    * families
171    */
172   public void addPeer(String id, ReplicationPeerConfig peerConfig,
173       Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
174     this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
175   }
176 
177   @VisibleForTesting
178   static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
179     String tableCfsStr = null;
180     if (tableCfs != null) {
181       // Format: table1:cf1,cf2;table2:cfA,cfB;table3
182       StringBuilder builder = new StringBuilder();
183       for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
184         if (builder.length() > 0) {
185           builder.append(";");
186         }
187         builder.append(entry.getKey());
188         if (entry.getValue() != null && !entry.getValue().isEmpty()) {
189           builder.append(":");
190           builder.append(StringUtils.join(entry.getValue(), ","));
191         }
192       }
193       tableCfsStr = builder.toString();
194     }
195     return tableCfsStr;
196   }
197 
198   /**
199    * Removes a peer cluster and stops the replication to it.
200    * @param id a short name that identifies the cluster
201    */
202   public void removePeer(String id) throws ReplicationException {
203     this.replicationPeers.removePeer(id);
204   }
205 
206   /**
207    * Restart the replication stream to the specified peer.
208    * @param id a short name that identifies the cluster
209    */
210   public void enablePeer(String id) throws ReplicationException {
211     this.replicationPeers.enablePeer(id);
212   }
213 
214   /**
215    * Stop the replication stream to the specified peer.
216    * @param id a short name that identifies the cluster
217    */
218   public void disablePeer(String id) throws ReplicationException {
219     this.replicationPeers.disablePeer(id);
220   }
221 
222   /**
223    * Get the number of slave clusters the local cluster has.
224    * @return number of slave clusters
225    */
226   public int getPeersCount() {
227     return this.replicationPeers.getAllPeerIds().size();
228   }
229 
230   /**
231    * Map of this cluster's peers for display.
232    * @return A map of peer ids to peer cluster keys
233    * @deprecated use {@link #listPeerConfigs()}
234    */
235   @Deprecated
236   public Map<String, String> listPeers() {
237     Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
238     Map<String, String> ret = new HashMap<String, String>(peers.size());
239 
240     for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
241       ret.put(entry.getKey(), entry.getValue().getClusterKey());
242     }
243     return ret;
244   }
245 
246   public Map<String, ReplicationPeerConfig> listPeerConfigs() {
247     return this.replicationPeers.getAllPeerConfigs();
248   }
249 
250   public ReplicationPeerConfig getPeerConfig(String id) throws ReplicationException {
251     return this.replicationPeers.getReplicationPeerConfig(id);
252   }
253 
254   /**
255    * Get the replicable table-cf config of the specified peer.
256    * @param id a short name that identifies the cluster
257    */
258   public String getPeerTableCFs(String id) throws ReplicationException {
259     return this.replicationPeers.getPeerTableCFsConfig(id);
260   }
261 
262   /**
263    * Set the replicable table-cf config of the specified peer
264    * @param id a short name that identifies the cluster
265    * @deprecated use {@link #setPeerTableCFs(String, Map)}
266    */
267   @Deprecated
268   public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
269     this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
270   }
271 
272   /**
273    * Set the replicable table-cf config of the specified peer
274    * @param id a short name that identifies the cluster
275    * @param tableCfs the table and column-family list which will be replicated for this peer.
276    * A map from tableName to column family names. An empty collection can be passed
277    * to indicate replicating all column families. Pass null for replicating all table and column
278    * families
279    */
280   public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
281       throws ReplicationException {
282     this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
283   }
284 
285   /**
286    * Get the state of the specified peer cluster
287    * @param id String format of the Short name that identifies the peer,
288    * an IllegalArgumentException is thrown if it doesn't exist
289    * @return true if replication is enabled to that peer, false if it isn't
290    */
291   public boolean getPeerState(String id) throws ReplicationException {
292     return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
293   }
294 
295   @Override
296   public void close() throws IOException {
297     if (this.zkw != null) {
298       this.zkw.close();
299     }
300     if (this.connection != null) {
301       this.connection.close();
302     }
303   }
304 
305 
306   /**
307    * Find all column families that are replicated from this cluster
308    * @return the full list of the replicated column families of this cluster as:
309    *        tableName, family name, replicationType
310    *
311    * Currently replicationType is Global. In the future, more replication
312    * types may be extended here. For example
313    *  1) the replication may only apply to selected peers instead of all peers
314    *  2) the replicationType may indicate the host Cluster servers as Slave
315    *     for the table:columnFam.
316    */
317   public List<HashMap<String, String>> listReplicated() throws IOException {
318     List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
319     HTableDescriptor[] tables = this.connection.listTables();
320 
321     for (HTableDescriptor table : tables) {
322       HColumnDescriptor[] columns = table.getColumnFamilies();
323       String tableName = table.getNameAsString();
324       for (HColumnDescriptor column : columns) {
325         if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
326           // At this moment, the columfam is replicated to all peers
327           HashMap<String, String> replicationEntry = new HashMap<String, String>();
328           replicationEntry.put(TNAME, tableName);
329           replicationEntry.put(CFNAME, column.getNameAsString());
330           replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
331           replicationColFams.add(replicationEntry);
332         }
333       }
334     }
335 
336     return replicationColFams;
337   }
338 }