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.replication;
21  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.hbase.HConstants;
26  import org.apache.hadoop.hbase.HServerAddress;
27  import org.apache.hadoop.hbase.util.Bytes;
28  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
29  import org.apache.hadoop.util.StringUtils;
30  import org.apache.zookeeper.KeeperException;
31  import org.apache.zookeeper.WatchedEvent;
32  import org.apache.zookeeper.Watcher;
33  
34  import java.io.IOException;
35  import java.util.ArrayList;
36  import java.util.HashMap;
37  import java.util.List;
38  import java.util.Map;
39  import java.util.SortedMap;
40  import java.util.TreeMap;
41  import java.util.SortedSet;
42  import java.util.TreeSet;
43  import java.util.concurrent.atomic.AtomicBoolean;
44  
45  /**
46   * This class serves as a helper for all things related to zookeeper
47   * in replication.
48   * <p/>
49   * The layout looks something like this under zookeeper.znode.parent
50   * for the master cluster:
51   * <p/>
52   * <pre>
53   * replication/
54   *  master     {contains a full cluster address}
55   *  state      {contains true or false}
56   *  clusterId  {contains a byte}
57   *  peers/
58   *    1/   {contains a full cluster address}
59   *    2/
60   *    ...
61   *  rs/ {lists all RS that replicate}
62   *    startcode1/ {lists all peer clusters}
63   *      1/ {lists hlogs to process}
64   *        10.10.1.76%3A53488.123456789 {contains nothing or a position}
65   *        10.10.1.76%3A53488.123456790
66   *        ...
67   *      2/
68   *      ...
69   *    startcode2/
70   *    ...
71   * </pre>
72   */
73  public class ReplicationZookeeperWrapper {
74  
75    private static final Log LOG =
76        LogFactory.getLog(ReplicationZookeeperWrapper.class);
77    // Name of znode we use to lock when failover
78    private final static String RS_LOCK_ZNODE = "lock";
79    // Our handle on zookeeper
80    private final ZooKeeperWrapper zookeeperWrapper;
81    // Map of addresses of peer clusters with their ZKW
82    private final Map<String, ZooKeeperWrapper> peerClusters;
83    // Path to the root replication znode
84    private final String replicationZNode;
85    // Path to the peer clusters znode
86    private final String peersZNode;
87    // Path to the znode that contains all RS that replicates
88    private final String rsZNode;
89    // Path to this region server's name under rsZNode
90    private final String rsServerNameZnode;
91    // Name node if the replicationState znode
92    private final String replicationStateNodeName;
93    // If this RS is part of a master cluster
94    private final boolean replicationMaster;
95    private final Configuration conf;
96    // Is this cluster replicating at the moment?
97    private final AtomicBoolean replicating;
98    // Byte (stored as string here) that identifies this cluster
99    private final String clusterId;
100 
101   /**
102    * Constructor used by region servers, connects to the peer cluster right away.
103    *
104    * @param zookeeperWrapper zkw to wrap
105    * @param conf             conf to use
106    * @param replicating    atomic boolean to start/stop replication
107    * @param rsName      the name of this region server, null if
108    *                         using RZH only to use the helping methods
109    * @throws IOException
110    */
111   public ReplicationZookeeperWrapper(
112       ZooKeeperWrapper zookeeperWrapper, Configuration conf,
113       final AtomicBoolean replicating, String rsName) throws IOException {
114     this.zookeeperWrapper = zookeeperWrapper;
115     this.conf = conf;
116     String replicationZNodeName =
117         conf.get("zookeeper.znode.replication", "replication");
118     String peersZNodeName =
119         conf.get("zookeeper.znode.replication.peers", "peers");
120     String repMasterZNodeName =
121         conf.get("zookeeper.znode.replication.master", "master");
122     this.replicationStateNodeName =
123         conf.get("zookeeper.znode.replication.state", "state");
124     String clusterIdZNodeName =
125         conf.get("zookeeper.znode.replication.clusterId", "clusterId");
126     String rsZNodeName =
127         conf.get("zookeeper.znode.replication.rs", "rs");
128     String thisCluster = this.conf.get(HConstants.ZOOKEEPER_QUORUM) + ":" +
129           this.conf.get("hbase.zookeeper.property.clientPort") + ":" +
130           this.conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
131 
132     this.peerClusters = new HashMap<String, ZooKeeperWrapper>();
133     this.replicationZNode = zookeeperWrapper.getZNode(
134         zookeeperWrapper.getParentZNode(), replicationZNodeName);
135     this.peersZNode =
136         zookeeperWrapper.getZNode(replicationZNode, peersZNodeName);
137     this.rsZNode =
138         zookeeperWrapper.getZNode(replicationZNode, rsZNodeName);
139 
140     this.replicating = replicating;
141     setReplicating();
142     String idResult = Bytes.toString(
143         this.zookeeperWrapper.getData(this.replicationZNode,
144         clusterIdZNodeName));
145     this.clusterId =
146         idResult == null ?
147             Byte.toString(HConstants.DEFAULT_CLUSTER_ID) : idResult;
148     String address = Bytes.toString(
149         this.zookeeperWrapper.getData(this.replicationZNode,
150           repMasterZNodeName));
151     this.replicationMaster = thisCluster.equals(address);
152     LOG.info("This cluster (" + thisCluster + ") is a "
153           + (this.replicationMaster ? "master" : "slave") + " for replication" +
154           ", compared with (" + address + ")");
155     if (rsName != null) {
156       this.rsServerNameZnode =
157           this.zookeeperWrapper.getZNode(rsZNode, rsName);
158       List<String> znodes = this.zookeeperWrapper.listZnodes(this.peersZNode,
159           new ReplicationStatusWatcher());
160       if (znodes != null) {
161         for (String znode : znodes) {
162           connectToPeer(znode);
163         }
164       }
165     } else {
166       this.rsServerNameZnode = null;
167     }
168 
169   }
170 
171   /**
172    * Returns all region servers from given peer
173    *
174    * @param peerClusterId (byte) the cluster to interrogate
175    * @return addresses of all region servers
176    */
177   public List<HServerAddress> getPeersAddresses(String peerClusterId) {
178     if (this.peerClusters.size() == 0) {
179       return new ArrayList<HServerAddress>(0);
180     }
181     ZooKeeperWrapper zkw = this.peerClusters.get(peerClusterId);
182     return zkw == null?
183         new ArrayList<HServerAddress>(0) : zkw.scanRSDirectory();
184   }
185 
186   /**
187    * This method connects this cluster to another one and registers it
188    * in this region server's replication znode
189    * @param peerId id of the peer cluster
190    */
191   private void connectToPeer(String peerId) throws IOException {
192     String[] ensemble =
193         Bytes.toString(this.zookeeperWrapper.getData(this.peersZNode, peerId)).
194             split(":");
195     if (ensemble.length != 3) {
196       throw new IllegalArgumentException("Wrong format of cluster address: " +
197           this.zookeeperWrapper.getData(this.peersZNode, peerId));
198     }
199     Configuration otherConf = new Configuration(this.conf);
200     otherConf.set(HConstants.ZOOKEEPER_QUORUM, ensemble[0]);
201     otherConf.set("hbase.zookeeper.property.clientPort", ensemble[1]);
202     otherConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, ensemble[2]);
203     ZooKeeperWrapper zkw = ZooKeeperWrapper.createInstance(otherConf,
204         "connection to cluster: " + peerId);
205     zkw.registerListener(new ReplicationStatusWatcher());
206     this.peerClusters.put(peerId, zkw);
207     this.zookeeperWrapper.ensureExists(this.zookeeperWrapper.getZNode(
208         this.rsServerNameZnode, peerId));
209     LOG.info("Added new peer cluster " + StringUtils.arrayToString(ensemble));
210   }
211 
212   /**
213    * This reads the state znode for replication and sets the atomic boolean
214    */
215   private void setReplicating() {
216     String value = Bytes.toString(this.zookeeperWrapper.getDataAndWatch(
217         this.replicationZNode, this.replicationStateNodeName,
218         new ReplicationStatusWatcher()));
219     if (value != null) {
220       this.replicating.set(value.equals("true"));
221       LOG.info("Replication is now " + (this.replicating.get() ?
222           "started" : "stopped"));
223     }
224   }
225 
226   /**
227    * Add a new log to the list of hlogs in zookeeper
228    * @param filename name of the hlog's znode
229    * @param clusterId name of the cluster's znode
230    */
231   public void addLogToList(String filename, String clusterId) {
232     try {
233       this.zookeeperWrapper.writeZNode(
234           this.zookeeperWrapper.getZNode(
235               this.rsServerNameZnode, clusterId), filename, "");
236     } catch (InterruptedException e) {
237       LOG.error(e);
238     } catch (KeeperException e) {
239       LOG.error(e);
240     }
241   }
242 
243   /**
244    * Remove a log from the list of hlogs in zookeeper
245    * @param filename name of the hlog's znode
246    * @param clusterId name of the cluster's znode
247    */
248   public void removeLogFromList(String filename, String clusterId) {
249     try {
250       this.zookeeperWrapper.deleteZNode(
251           this.zookeeperWrapper.getZNode(this.rsServerNameZnode,
252               this.zookeeperWrapper.getZNode(clusterId, filename)));
253     } catch (InterruptedException e) {
254       LOG.error(e);
255     } catch (KeeperException e) {
256       LOG.error(e);
257     }
258   }
259 
260   /**
261    * Set the current position of the specified cluster in the current hlog
262    * @param filename filename name of the hlog's znode
263    * @param clusterId clusterId name of the cluster's znode
264    * @param position the position in the file
265    * @throws IOException
266    */
267   public void writeReplicationStatus(String filename, String clusterId,
268                                      long position) {
269     try {
270       String clusterZNode = this.zookeeperWrapper.getZNode(
271           this.rsServerNameZnode, clusterId);
272       this.zookeeperWrapper.writeZNode(clusterZNode, filename,
273           Long.toString(position));
274     } catch (InterruptedException e) {
275       LOG.error(e);
276     } catch (KeeperException e) {
277       LOG.error(e);
278     }
279   }
280 
281   /**
282    * Get a list of all the other region servers in this cluster
283    * and set a watch
284    * @param watch the watch to set
285    * @return a list of server nanes
286    */
287   public List<String> getRegisteredRegionServers(Watcher watch) {
288     return this.zookeeperWrapper.listZnodes(
289         this.zookeeperWrapper.getRsZNode(), watch);
290   }
291 
292   /**
293    * Get the list of the replicators that have queues, they can be alive, dead
294    * or simply from a previous run
295    * @param watch the watche to set
296    * @return a list of server names
297    */
298   public List<String> getListOfReplicators(Watcher watch) {
299     return this.zookeeperWrapper.listZnodes(rsZNode, watch);
300   }
301 
302   /**
303    * Get the list of peer clusters for the specified server names
304    * @param rs server names of the rs
305    * @param watch the watch to set
306    * @return a list of peer cluster
307    */
308   public List<String> getListPeersForRS(String rs, Watcher watch) {
309     return this.zookeeperWrapper.listZnodes(
310         zookeeperWrapper.getZNode(rsZNode, rs), watch);
311   }
312 
313   /**
314    * Get the list of hlogs for the specified region server and peer cluster
315    * @param rs server names of the rs
316    * @param id peer cluster
317    * @param watch the watch to set
318    * @return a list of hlogs
319    */
320   public List<String> getListHLogsForPeerForRS(String rs, String id, Watcher watch) {
321     return this.zookeeperWrapper.listZnodes(
322         zookeeperWrapper.getZNode(zookeeperWrapper.getZNode(rsZNode, rs), id), watch);
323   }
324 
325   /**
326    * Try to set a lock in another server's znode.
327    * @param znode the server names of the other server
328    * @return true if the lock was acquired, false in every other cases
329    */
330   public boolean lockOtherRS(String znode) {
331     try {
332       this.zookeeperWrapper.writeZNode(
333           this.zookeeperWrapper.getZNode(this.rsZNode, znode),
334           RS_LOCK_ZNODE, rsServerNameZnode, true);
335 
336     } catch (InterruptedException e) {
337       LOG.error(e);
338       return false;
339     } catch (KeeperException e) {
340       LOG.debug("Won't lock " + znode + " because " + e.getMessage());
341       // TODO see if the other still exists!!
342       return false;
343     }
344     return true;
345   }
346 
347   /**
348    * This methods copies all the hlogs queues from another region server
349    * and returns them all sorted per peer cluster (appended with the dead
350    * server's znode)
351    * @param znode server names to copy
352    * @return all hlogs for all peers of that cluster, null if an error occurred
353    */
354   public SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
355     // TODO this method isn't atomic enough, we could start copying and then
356     // TODO fail for some reason and we would end up with znodes we don't want.
357     SortedMap<String,SortedSet<String>> queues =
358         new TreeMap<String,SortedSet<String>>();
359     try {
360       String nodePath = this.zookeeperWrapper.getZNode(rsZNode, znode);
361       List<String> clusters = this.zookeeperWrapper.listZnodes(nodePath, null);
362       // We have a lock znode in there, it will count as one.
363       if (clusters == null || clusters.size() <= 1) {
364         return queues;
365       }
366       // The lock isn't a peer cluster, remove it
367       clusters.remove(RS_LOCK_ZNODE);
368       for (String cluster : clusters) {
369         // We add the name of the recovered RS to the new znode, we can even
370         // do that for queues that were recovered 10 times giving a znode like
371         // number-startcode-number-otherstartcode-number-anotherstartcode-etc
372         String newCluster = cluster+"-"+znode;
373         String newClusterZnode =
374             this.zookeeperWrapper.getZNode(rsServerNameZnode, newCluster);
375         this.zookeeperWrapper.ensureExists(newClusterZnode);
376         String clusterPath = this.zookeeperWrapper.getZNode(nodePath, cluster);
377         List<String> hlogs = this.zookeeperWrapper.listZnodes(clusterPath, null);
378         // That region server didn't have anything to replicate for this cluster
379         if (hlogs == null || hlogs.size() == 0) {
380           continue;
381         }
382         SortedSet<String> logQueue = new TreeSet<String>();
383         queues.put(newCluster, logQueue);
384         for (String hlog : hlogs) {
385           String position = Bytes.toString(
386               this.zookeeperWrapper.getData(clusterPath, hlog));
387           LOG.debug("Creating " + hlog + " with data " + position);
388           this.zookeeperWrapper.writeZNode(newClusterZnode, hlog, position);
389           logQueue.add(hlog);
390         }
391       }
392     } catch (InterruptedException e) {
393       LOG.warn(e);
394       return null;
395     } catch (KeeperException e) {
396       LOG.warn(e);
397       return null;
398     }
399     return queues;
400   }
401 
402   /**
403    * Delete a complete queue of hlogs
404    * @param peerZnode znode of the peer cluster queue of hlogs to delete
405    */
406   public void deleteSource(String peerZnode) {
407     try {
408       this.zookeeperWrapper.deleteZNode(
409           this.zookeeperWrapper.getZNode(rsServerNameZnode, peerZnode), true);
410     } catch (InterruptedException e) {
411       LOG.error(e);
412     } catch (KeeperException e) {
413       LOG.error(e);
414     }
415   }
416 
417   /**
418    * Recursive deletion of all znodes in specified rs' znode
419    * @param znode
420    */
421   public void deleteRsQueues(String znode) {
422     try {
423       this.zookeeperWrapper.deleteZNode(
424           this.zookeeperWrapper.getZNode(rsZNode, znode), true);
425     } catch (InterruptedException e) {
426       LOG.error(e);
427     } catch (KeeperException e) {
428       LOG.error(e);
429     }
430   }
431 
432   /**
433    * Delete this cluster's queues
434    */
435   public void deleteOwnRSZNode() {
436     deleteRsQueues(this.rsServerNameZnode);
437   }
438 
439   /**
440    * Get the position of the specified hlog in the specified peer znode
441    * @param peerId znode of the peer cluster
442    * @param hlog name of the hlog
443    * @return the position in that hlog
444    */
445   public long getHLogRepPosition(String peerId, String hlog) {
446     String clusterZnode =
447         this.zookeeperWrapper.getZNode(rsServerNameZnode, peerId);
448     String data = Bytes.toString(
449         this.zookeeperWrapper.getData(clusterZnode, hlog));
450     return data == null || data.length() == 0 ? 0 : Long.parseLong(data);
451   }
452 
453   /**
454    * Tells if this cluster replicates or not
455    *
456    * @return if this is a master
457    */
458   public boolean isReplicationMaster() {
459     return this.replicationMaster;
460   }
461 
462   /**
463    * Get the identification of the cluster
464    *
465    * @return the id for the cluster
466    */
467   public String getClusterId() {
468     return this.clusterId;
469   }
470 
471   /**
472    * Get a map of all peer clusters
473    * @return map of peer cluster, zk address to ZKW
474    */
475   public Map<String, ZooKeeperWrapper> getPeerClusters() {
476     return this.peerClusters;
477   }
478 
479   /**
480    * Watcher for the status of the replication
481    */
482   public class ReplicationStatusWatcher implements Watcher {
483     @Override
484     public void process(WatchedEvent watchedEvent) {
485       Event.EventType type = watchedEvent.getType();
486       LOG.info("Got event " + type + " with path " + watchedEvent.getPath());
487       if (type.equals(Event.EventType.NodeDataChanged)) {
488         setReplicating();
489       }
490     }
491   }
492 
493 }