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.HashSet;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Map.Entry;
30  import java.util.Set;
31  
32  import org.apache.commons.lang.StringUtils;
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.hbase.classification.InterfaceStability;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.hbase.Abortable;
39  import org.apache.hadoop.hbase.HColumnDescriptor;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.HTableDescriptor;
42  import org.apache.hadoop.hbase.TableName;
43  import org.apache.hadoop.hbase.TableNotFoundException;
44  import org.apache.hadoop.hbase.classification.InterfaceAudience;
45  import org.apache.hadoop.hbase.classification.InterfaceStability;
46  import org.apache.hadoop.hbase.client.Admin;
47  import org.apache.hadoop.hbase.client.HBaseAdmin;
48  import org.apache.hadoop.hbase.client.Connection;
49  import org.apache.hadoop.hbase.client.ConnectionFactory;
50  import org.apache.hadoop.hbase.client.RegionLocator;
51  import org.apache.hadoop.hbase.replication.ReplicationException;
52  import org.apache.hadoop.hbase.replication.ReplicationFactory;
53  import org.apache.hadoop.hbase.replication.ReplicationPeer;
54  import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
55  import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
56  import org.apache.hadoop.hbase.replication.ReplicationPeers;
57  import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
58  import org.apache.hadoop.hbase.util.Pair;
59  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
60  import org.apache.zookeeper.KeeperException;
61  import org.apache.zookeeper.data.Stat;
62  
63  import com.google.common.annotations.VisibleForTesting;
64  import com.google.common.collect.Lists;
65  
66  /**
67   * <p>
68   * This class provides the administrative interface to HBase cluster
69   * replication. In order to use it, the cluster and the client using
70   * ReplicationAdmin must be configured with <code>hbase.replication</code>
71   * set to true.
72   * </p>
73   * <p>
74   * Adding a new peer results in creating new outbound connections from every
75   * region server to a subset of region servers on the slave cluster. Each
76   * new stream of replication will start replicating from the beginning of the
77   * current WAL, meaning that edits from that past will be replicated.
78   * </p>
79   * <p>
80   * Removing a peer is a destructive and irreversible operation that stops
81   * all the replication streams for the given cluster and deletes the metadata
82   * used to keep track of the replication state.
83   * </p>
84   * <p>
85   * To see which commands are available in the shell, type
86   * <code>replication</code>.
87   * </p>
88   */
89  @InterfaceAudience.Public
90  @InterfaceStability.Evolving
91  public class ReplicationAdmin implements Closeable {
92    private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
93  
94    public static final String TNAME = "tableName";
95    public static final String CFNAME = "columnFamlyName";
96  
97    // only Global for now, can add other type
98    // such as, 1) no global replication, or 2) the table is replicated to this cluster, etc.
99    public static final String REPLICATIONTYPE = "replicationType";
100   public static final String REPLICATIONGLOBAL = Integer
101       .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
102 
103   private final Connection connection;
104   // TODO: replication should be managed by master. All the classes except ReplicationAdmin should
105   // be moved to hbase-server. Resolve it in HBASE-11392.
106   private final ReplicationQueuesClient replicationQueuesClient;
107   private final ReplicationPeers replicationPeers;
108   /**
109    * A watcher used by replicationPeers and replicationQueuesClient. Keep reference so can dispose
110    * on {@link #close()}.
111    */
112   private final ZooKeeperWatcher zkw;
113 
114   /**
115    * Constructor that creates a connection to the local ZooKeeper ensemble.
116    * @param conf Configuration to use
117    * @throws IOException if an internal replication error occurs
118    * @throws RuntimeException if replication isn't enabled.
119    */
120   public ReplicationAdmin(Configuration conf) throws IOException {
121     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
122         HConstants.REPLICATION_ENABLE_DEFAULT)) {
123       throw new RuntimeException("hbase.replication isn't true, please " +
124           "enable it in order to use replication");
125     }
126     this.connection = ConnectionFactory.createConnection(conf);
127     try {
128       zkw = createZooKeeperWatcher();
129       try {
130         this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
131         this.replicationPeers.init();
132         this.replicationQueuesClient =
133             ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
134         this.replicationQueuesClient.init();
135       } catch (Exception exception) {
136         if (zkw != null) {
137           zkw.close();
138         }
139         throw exception;
140       }
141     } catch (Exception exception) {
142       if (connection != null) {
143         connection.close();
144       }
145       if (exception instanceof IOException) {
146         throw (IOException) exception;
147       } else if (exception instanceof RuntimeException) {
148         throw (RuntimeException) exception;
149       } else {
150         throw new IOException("Error initializing the replication admin client.", exception);
151       }
152     }
153   }
154 
155   private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
156     // This Abortable doesn't 'abort'... it just logs.
157     return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
158       @Override
159       public void abort(String why, Throwable e) {
160         LOG.error(why, e);
161         // We used to call system.exit here but this script can be embedded by other programs that
162         // want to do replication stuff... so inappropriate calling System.exit. Just log for now.
163       }
164 
165       @Override
166       public boolean isAborted() {
167         return false;
168       }
169     });
170   }
171 
172   /**
173    * Add a new peer cluster to replicate to.
174    * @param id a short name that identifies the cluster
175    * @param clusterKey the concatenation of the slave cluster's
176    * <code>hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent</code>
177    * @throws IllegalStateException if there's already one slave since
178    * multi-slave isn't supported yet.
179    * @deprecated Use addPeer(String, ReplicationPeerConfig, Map) instead.
180    */
181   @Deprecated
182   public void addPeer(String id, String clusterKey) throws ReplicationException {
183     this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
184   }
185 
186   @Deprecated
187   public void addPeer(String id, String clusterKey, String tableCFs)
188     throws ReplicationException {
189     this.replicationPeers.addPeer(id,
190       new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
191   }
192 
193   /**
194    * Add a new remote slave cluster for replication.
195    * @param id a short name that identifies the cluster
196    * @param peerConfig configuration for the replication slave cluster
197    * @param tableCfs the table and column-family list which will be replicated for this peer.
198    * A map from tableName to column family names. An empty collection can be passed
199    * to indicate replicating all column families. Pass null for replicating all table and column
200    * families
201    */
202   public void addPeer(String id, ReplicationPeerConfig peerConfig,
203       Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
204     this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
205   }
206 
207   public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
208     if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
209       return null;
210     }
211 
212     Map<TableName, List<String>> tableCFsMap = null;
213     // TODO: This should be a PB object rather than a String to be parsed!! See HBASE-11393
214     // parse out (table, cf-list) pairs from tableCFsConfig
215     // format: "table1:cf1,cf2;table2:cfA,cfB"
216     String[] tables = tableCFsConfig.split(";");
217     for (String tab : tables) {
218       // 1 ignore empty table config
219       tab = tab.trim();
220       if (tab.length() == 0) {
221         continue;
222       }
223       // 2 split to "table" and "cf1,cf2"
224       //   for each table: "table:cf1,cf2" or "table"
225       String[] pair = tab.split(":");
226       String tabName = pair[0].trim();
227       if (pair.length > 2 || tabName.length() == 0) {
228         LOG.error("ignore invalid tableCFs setting: " + tab);
229         continue;
230       }
231 
232       // 3 parse "cf1,cf2" part to List<cf>
233       List<String> cfs = null;
234       if (pair.length == 2) {
235         String[] cfsList = pair[1].split(",");
236         for (String cf : cfsList) {
237           String cfName = cf.trim();
238           if (cfName.length() > 0) {
239             if (cfs == null) {
240               cfs = new ArrayList<String>();
241             }
242             cfs.add(cfName);
243           }
244         }
245       }
246 
247       // 4 put <table, List<cf>> to map
248       if (tableCFsMap == null) {
249         tableCFsMap = new HashMap<TableName, List<String>>();
250       }
251       tableCFsMap.put(TableName.valueOf(tabName), cfs);
252     }
253     return tableCFsMap;
254   }
255 
256   @VisibleForTesting
257   static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
258     String tableCfsStr = null;
259     if (tableCfs != null) {
260       // Format: table1:cf1,cf2;table2:cfA,cfB;table3
261       StringBuilder builder = new StringBuilder();
262       for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
263         if (builder.length() > 0) {
264           builder.append(";");
265         }
266         builder.append(entry.getKey());
267         if (entry.getValue() != null && !entry.getValue().isEmpty()) {
268           builder.append(":");
269           builder.append(StringUtils.join(entry.getValue(), ","));
270         }
271       }
272       tableCfsStr = builder.toString();
273     }
274     return tableCfsStr;
275   }
276 
277   /**
278    * Removes a peer cluster and stops the replication to it.
279    * @param id a short name that identifies the cluster
280    */
281   public void removePeer(String id) throws ReplicationException {
282     this.replicationPeers.removePeer(id);
283   }
284 
285   /**
286    * Restart the replication stream to the specified peer.
287    * @param id a short name that identifies the cluster
288    */
289   public void enablePeer(String id) throws ReplicationException {
290     this.replicationPeers.enablePeer(id);
291   }
292 
293   /**
294    * Stop the replication stream to the specified peer.
295    * @param id a short name that identifies the cluster
296    */
297   public void disablePeer(String id) throws ReplicationException {
298     this.replicationPeers.disablePeer(id);
299   }
300 
301   /**
302    * Get the number of slave clusters the local cluster has.
303    * @return number of slave clusters
304    */
305   public int getPeersCount() {
306     return this.replicationPeers.getAllPeerIds().size();
307   }
308 
309   /**
310    * Map of this cluster's peers for display.
311    * @return A map of peer ids to peer cluster keys
312    * @deprecated use {@link #listPeerConfigs()}
313    */
314   @Deprecated
315   public Map<String, String> listPeers() {
316     Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
317     Map<String, String> ret = new HashMap<String, String>(peers.size());
318 
319     for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
320       ret.put(entry.getKey(), entry.getValue().getClusterKey());
321     }
322     return ret;
323   }
324 
325   public Map<String, ReplicationPeerConfig> listPeerConfigs() {
326     return this.replicationPeers.getAllPeerConfigs();
327   }
328 
329   public ReplicationPeerConfig getPeerConfig(String id) throws ReplicationException {
330     return this.replicationPeers.getReplicationPeerConfig(id);
331   }
332 
333   /**
334    * Get the replicable table-cf config of the specified peer.
335    * @param id a short name that identifies the cluster
336    */
337   public String getPeerTableCFs(String id) throws ReplicationException {
338     return this.replicationPeers.getPeerTableCFsConfig(id);
339   }
340 
341   /**
342    * Set the replicable table-cf config of the specified peer
343    * @param id a short name that identifies the cluster
344    * @deprecated use {@link #setPeerTableCFs(String, Map)}
345    */
346   @Deprecated
347   public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
348     this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
349   }
350 
351   /**
352    * Append the replicable table-cf config of the specified peer
353    * @param id a short that identifies the cluster
354    * @param tableCfs table-cfs config str
355    * @throws KeeperException
356    */
357   public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException {
358     appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs));
359   }
360 
361   /**
362    * Append the replicable table-cf config of the specified peer
363    * @param id a short that identifies the cluster
364    * @param tableCfs A map from tableName to column family names
365    * @throws KeeperException
366    */
367   public void appendPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
368       throws ReplicationException {
369     if (tableCfs == null) {
370       throw new ReplicationException("tableCfs is null");
371     }
372     Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
373     if (preTableCfs == null) {
374       setPeerTableCFs(id, tableCfs);
375       return;
376     }
377 
378     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
379       TableName table = entry.getKey();
380       Collection<String> appendCfs = entry.getValue();
381       if (preTableCfs.containsKey(table)) {
382         List<String> cfs = preTableCfs.get(table);
383         if (cfs == null || appendCfs == null) {
384           preTableCfs.put(table, null);
385         } else {
386           Set<String> cfSet = new HashSet<String>(cfs);
387           cfSet.addAll(appendCfs);
388           preTableCfs.put(table, Lists.newArrayList(cfSet));
389         }
390       } else {
391         if (appendCfs == null || appendCfs.isEmpty()) {
392           preTableCfs.put(table, null);
393         } else {
394           preTableCfs.put(table, Lists.newArrayList(appendCfs));
395         }
396       }
397     }
398     setPeerTableCFs(id, preTableCfs);
399   }
400 
401   /**
402    * Remove some table-cfs from table-cfs config of the specified peer
403    * @param id a short name that identifies the cluster
404    * @param tableCf table-cfs config str
405    * @throws ReplicationException
406    */
407   public void removePeerTableCFs(String id, String tableCf) throws ReplicationException {
408     removePeerTableCFs(id, parseTableCFsFromConfig(tableCf));
409   }
410 
411   /**
412    * Remove some table-cfs from config of the specified peer
413    * @param id a short name that identifies the cluster
414    * @param tableCfs A map from tableName to column family names
415    * @throws ReplicationException
416    */
417   public void removePeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
418       throws ReplicationException {
419     if (tableCfs == null) {
420       throw new ReplicationException("tableCfs is null");
421     }
422 
423     Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
424     if (preTableCfs == null) {
425       throw new ReplicationException("Table-Cfs for peer" + id + " is null");
426     }
427     for (Map.Entry<TableName, ? extends Collection<String>> entry: tableCfs.entrySet()) {
428       TableName table = entry.getKey();
429       Collection<String> removeCfs = entry.getValue();
430       if (preTableCfs.containsKey(table)) {
431         List<String> cfs = preTableCfs.get(table);
432         if (cfs == null && removeCfs == null) {
433           preTableCfs.remove(table);
434         } else if (cfs != null && removeCfs != null) {
435           Set<String> cfSet = new HashSet<String>(cfs);
436           cfSet.removeAll(removeCfs);
437           if (cfSet.isEmpty()) {
438             preTableCfs.remove(table);
439           } else {
440             preTableCfs.put(table, Lists.newArrayList(cfSet));
441           }
442         } else if (cfs == null && removeCfs != null) {
443           throw new ReplicationException("Cannot remove cf of table: " + table
444               + " which doesn't specify cfs from table-cfs config in peer: " + id);
445         } else if (cfs != null && removeCfs == null) {
446           throw new ReplicationException("Cannot remove table: " + table
447               + " which has specified cfs from table-cfs config in peer: " + id);
448         }
449       } else {
450         throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id);
451       }
452     }
453     setPeerTableCFs(id, preTableCfs);
454   }
455 
456   /**
457    * Set the replicable table-cf config of the specified peer
458    * @param id a short name that identifies the cluster
459    * @param tableCfs the table and column-family list which will be replicated for this peer.
460    * A map from tableName to column family names. An empty collection can be passed
461    * to indicate replicating all column families. Pass null for replicating all table and column
462    * families
463    */
464   public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
465       throws ReplicationException {
466     this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
467   }
468 
469   /**
470    * Get the state of the specified peer cluster
471    * @param id String format of the Short name that identifies the peer,
472    * an IllegalArgumentException is thrown if it doesn't exist
473    * @return true if replication is enabled to that peer, false if it isn't
474    */
475   public boolean getPeerState(String id) throws ReplicationException {
476     return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
477   }
478 
479   @Override
480   public void close() throws IOException {
481     if (this.zkw != null) {
482       this.zkw.close();
483     }
484     if (this.connection != null) {
485       this.connection.close();
486     }
487   }
488 
489 
490   /**
491    * Find all column families that are replicated from this cluster
492    * @return the full list of the replicated column families of this cluster as:
493    *        tableName, family name, replicationType
494    *
495    * Currently replicationType is Global. In the future, more replication
496    * types may be extended here. For example
497    *  1) the replication may only apply to selected peers instead of all peers
498    *  2) the replicationType may indicate the host Cluster servers as Slave
499    *     for the table:columnFam.
500    */
501   public List<HashMap<String, String>> listReplicated() throws IOException {
502     List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
503 
504     Admin admin = connection.getAdmin();
505     HTableDescriptor[] tables;
506     try {
507       tables = admin.listTables();
508     } finally {
509       if (admin!= null) admin.close();
510     }
511 
512     for (HTableDescriptor table : tables) {
513       HColumnDescriptor[] columns = table.getColumnFamilies();
514       String tableName = table.getNameAsString();
515       for (HColumnDescriptor column : columns) {
516         if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
517           // At this moment, the columfam is replicated to all peers
518           HashMap<String, String> replicationEntry = new HashMap<String, String>();
519           replicationEntry.put(TNAME, tableName);
520           replicationEntry.put(CFNAME, column.getNameAsString());
521           replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
522           replicationColFams.add(replicationEntry);
523         }
524       }
525     }
526 
527     return replicationColFams;
528   }
529 
530   /**
531    * Enable a table's replication switch.
532    * @param tableName name of the table
533    * @throws IOException if a remote or network exception occurs
534    */
535   public void enableTableRep(final TableName tableName) throws IOException {
536     if (tableName == null) {
537       throw new IllegalArgumentException("Table name cannot be null");
538     }
539     try (Admin admin = this.connection.getAdmin()) {
540       if (!admin.tableExists(tableName)) {
541         throw new TableNotFoundException("Table '" + tableName.getNameAsString()
542             + "' does not exists.");
543       }
544     }
545     byte[][] splits = getTableSplitRowKeys(tableName);
546     checkAndSyncTableDescToPeers(tableName, splits);
547     setTableRep(tableName, true);
548   }
549 
550   /**
551    * Disable a table's replication switch.
552    * @param tableName name of the table
553    * @throws IOException if a remote or network exception occurs
554    */
555   public void disableTableRep(final TableName tableName) throws IOException {
556     if (tableName == null) {
557       throw new IllegalArgumentException("Table name is null");
558     }
559     try (Admin admin = this.connection.getAdmin()) {
560       if (!admin.tableExists(tableName)) {
561         throw new TableNotFoundException("Table '" + tableName.getNamespaceAsString()
562             + "' does not exists.");
563       }
564     }
565     setTableRep(tableName, false);
566   }
567 
568   /**
569    * Get the split row keys of table
570    * @param tableName table name
571    * @return array of split row keys
572    * @throws IOException
573    */
574   private byte[][] getTableSplitRowKeys(TableName tableName) throws IOException {
575     try (RegionLocator locator = connection.getRegionLocator(tableName);) {
576       byte[][] startKeys = locator.getStartKeys();
577       if (startKeys.length == 1) {
578         return null;
579       }
580       byte[][] splits = new byte[startKeys.length - 1][];
581       for (int i = 1; i < startKeys.length; i++) {
582         splits[i - 1] = startKeys[i];
583       }
584       return splits;
585     }
586   }
587 
588   /**
589    * Connect to peer and check the table descriptor on peer:
590    * <ol>
591    * <li>Create the same table on peer when not exist.</li>
592    * <li>Throw exception if the table exists on peer cluster but descriptors are not same.</li>
593    * </ol>
594    * @param tableName name of the table to sync to the peer
595    * @param splits table split keys
596    * @throws IOException
597    */
598   private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits)
599       throws IOException {
600     List<ReplicationPeer> repPeers = listValidReplicationPeers();
601     if (repPeers == null || repPeers.size() <= 0) {
602       throw new IllegalArgumentException("Found no peer cluster for replication.");
603     }
604     for (ReplicationPeer repPeer : repPeers) {
605       Configuration peerConf = repPeer.getConfiguration();
606       HTableDescriptor htd = null;
607       try (Connection conn = ConnectionFactory.createConnection(peerConf);
608           Admin admin = this.connection.getAdmin();
609           Admin repHBaseAdmin = conn.getAdmin()) {
610         htd = admin.getTableDescriptor(tableName);
611         HTableDescriptor peerHtd = null;
612         if (!repHBaseAdmin.tableExists(tableName)) {
613           repHBaseAdmin.createTable(htd, splits);
614         } else {
615           peerHtd = repHBaseAdmin.getTableDescriptor(tableName);
616           if (peerHtd == null) {
617             throw new IllegalArgumentException("Failed to get table descriptor for table "
618                 + tableName.getNameAsString() + " from peer cluster " + repPeer.getId());
619           } else if (!peerHtd.equals(htd)) {
620             throw new IllegalArgumentException("Table " + tableName.getNameAsString()
621                 + " exists in peer cluster " + repPeer.getId()
622                 + ", but the table descriptors are not same when comapred with source cluster."
623                 + " Thus can not enable the table's replication switch.");
624           }
625         }
626       }
627     }
628   }
629 
630   @VisibleForTesting
631   List<ReplicationPeer> listValidReplicationPeers() {
632     Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
633     if (peers == null || peers.size() <= 0) {
634       return null;
635     }
636     List<ReplicationPeer> validPeers = new ArrayList<ReplicationPeer>(peers.size());
637     for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
638       String peerId = peerEntry.getKey();
639       Stat s = null;
640       try {
641         Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
642         Configuration peerConf = pair.getSecond();
643         ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
644         s =
645             zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
646               null);
647         if (null == s) {
648           LOG.info(peerId + ' ' + pair.getFirst().getClusterKey() + " is invalid now.");
649           continue;
650         }
651         validPeers.add(peer);
652       } catch (ReplicationException e) {
653         LOG.warn("Failed to get valid replication peers. "
654             + "Error connecting to peer cluster with peerId=" + peerId);
655         LOG.debug("Failure details to get valid replication peers.", e);
656         continue;
657       } catch (KeeperException e) {
658         LOG.warn("Failed to get valid replication peers. KeeperException code="
659             + e.code().intValue());
660         LOG.debug("Failure details to get valid replication peers.", e);
661         continue;
662       } catch (InterruptedException e) {
663         LOG.warn("Failed to get valid replication peers due to InterruptedException.");
664         LOG.debug("Failure details to get valid replication peers.", e);
665         continue;
666       }
667     }
668     return validPeers;
669   }
670 
671   /**
672    * Set the table's replication switch if the table's replication switch is already not set.
673    * @param tableName name of the table
674    * @param isRepEnabled is replication switch enable or disable
675    * @throws IOException if a remote or network exception occurs
676    */
677   private void setTableRep(final TableName tableName, boolean isRepEnabled) throws IOException {
678     Admin admin = null;
679     try {
680       admin = this.connection.getAdmin();
681       HTableDescriptor htd = admin.getTableDescriptor(tableName);
682       if (isTableRepEnabled(htd) ^ isRepEnabled) {
683         boolean isOnlineSchemaUpdateEnabled =
684             this.connection.getConfiguration()
685                 .getBoolean("hbase.online.schema.update.enable", true);
686         if (!isOnlineSchemaUpdateEnabled) {
687           admin.disableTable(tableName);
688         }
689         for (HColumnDescriptor hcd : htd.getFamilies()) {
690           hcd.setScope(isRepEnabled ? HConstants.REPLICATION_SCOPE_GLOBAL
691               : HConstants.REPLICATION_SCOPE_LOCAL);
692         }
693         admin.modifyTable(tableName, htd);
694         if (!isOnlineSchemaUpdateEnabled) {
695           admin.enableTable(tableName);
696         }
697       }
698     } finally {
699       if (admin != null) {
700         try {
701           admin.close();
702         } catch (IOException e) {
703           LOG.warn("Failed to close admin connection.");
704           LOG.debug("Details on failure to close admin connection.", e);
705         }
706       }
707     }
708   }
709 
710   /**
711    * @param htd table descriptor details for the table to check
712    * @return true if table's replication switch is enabled
713    */
714   private boolean isTableRepEnabled(HTableDescriptor htd) {
715     for (HColumnDescriptor hcd : htd.getFamilies()) {
716       if (hcd.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
717         return false;
718       }
719     }
720     return true;
721   }
722 }