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  
21  package org.apache.hadoop.hbase.replication.regionserver;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.SortedMap;
28  import java.util.SortedSet;
29  import java.util.TreeSet;
30  import java.util.concurrent.atomic.AtomicBoolean;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.Stoppable;
38  import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
39  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
40  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41  import org.apache.zookeeper.KeeperException;
42  
43  /**
44   * This class is responsible to manage all the replication
45   * sources. There are two classes of sources:
46   * <li> Normal sources are persistent and one per peer cluster</li>
47   * <li> Old sources are recovered from a failed region server and our
48   * only goal is to finish replicating the HLog queue it had up in ZK</li>
49   *
50   * When a region server dies, this class uses a watcher to get notified and it
51   * tries to grab a lock in order to transfer all the queues in a local
52   * old source.
53   */
54  public class ReplicationSourceManager {
55    private static final Log LOG =
56        LogFactory.getLog(ReplicationSourceManager.class);
57    // List of all the sources that read this RS's logs
58    private final List<ReplicationSourceInterface> sources;
59    // List of all the sources we got from died RSs
60    private final List<ReplicationSourceInterface> oldsources;
61    // Indicates if we are currently replicating
62    private final AtomicBoolean replicating;
63    // Helper for zookeeper
64    private final ReplicationZookeeper zkHelper;
65    // All about stopping
66    private final Stoppable stopper;
67    // All logs we are currently trackign
68    private final SortedSet<String> hlogs;
69    private final Configuration conf;
70    private final FileSystem fs;
71    // The path to the latest log we saw, for new coming sources
72    private Path latestPath;
73    // List of all the other region servers in this cluster
74    private final List<String> otherRegionServers;
75    // Path to the hlogs directories
76    private final Path logDir;
77    // Path to the hlog archive
78    private final Path oldLogDir;
79  
80    /**
81     * Creates a replication manager and sets the watch on all the other
82     * registered region servers
83     * @param zkHelper the zk helper for replication
84     * @param conf the configuration to use
85     * @param stopper the stopper object for this region server
86     * @param fs the file system to use
87     * @param replicating the status of the replication on this cluster
88     * @param logDir the directory that contains all hlog directories of live RSs
89     * @param oldLogDir the directory where old logs are archived
90     */
91    public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
92                                    final Configuration conf,
93                                    final Stoppable stopper,
94                                    final FileSystem fs,
95                                    final AtomicBoolean replicating,
96                                    final Path logDir,
97                                    final Path oldLogDir) {
98      this.sources = new ArrayList<ReplicationSourceInterface>();
99      this.replicating = replicating;
100     this.zkHelper = zkHelper;
101     this.stopper = stopper;
102     this.hlogs = new TreeSet<String>();
103     this.oldsources = new ArrayList<ReplicationSourceInterface>();
104     this.conf = conf;
105     this.fs = fs;
106     this.logDir = logDir;
107     this.oldLogDir = oldLogDir;
108     this.zkHelper.registerRegionServerListener(
109         new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
110     List<String> otherRSs =
111         this.zkHelper.getRegisteredRegionServers();
112     this.zkHelper.registerRegionServerListener(
113         new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
114     this.zkHelper.listPeersIdsAndWatch();
115     this.otherRegionServers = otherRSs == null ? new ArrayList<String>() : otherRSs;
116   }
117 
118   /**
119    * Provide the id of the peer and a log key and this method will figure which
120    * hlog it belongs to and will log, for this region server, the current
121    * position. It will also clean old logs from the queue.
122    * @param log Path to the log currently being replicated from
123    * replication status in zookeeper. It will also delete older entries.
124    * @param id id of the peer cluster
125    * @param position current location in the log
126    * @param queueRecovered indicates if this queue comes from another region server
127    */
128   public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered) {
129     String key = log.getName();
130     LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
131     this.zkHelper.writeReplicationStatus(key.toString(), id, position);
132     synchronized (this.hlogs) {
133       if (!queueRecovered && this.hlogs.first() != key) {
134         SortedSet<String> hlogSet = this.hlogs.headSet(key);
135         LOG.info("Removing " + hlogSet.size() +
136             " logs in the list: " + hlogSet);
137         for (String hlog : hlogSet) {
138           this.zkHelper.removeLogFromList(hlog.toString(), id);
139         }
140         hlogSet.clear();
141       }
142     }
143   }
144 
145   /**
146    * Adds a normal source per registered peer cluster and tries to process all
147    * old region server hlog queues
148    */
149   public void init() throws IOException {
150     for (String id : this.zkHelper.getPeerClusters().keySet()) {
151       addSource(id);
152     }
153     List<String> currentReplicators = this.zkHelper.getListOfReplicators();
154     if (currentReplicators == null || currentReplicators.size() == 0) {
155       return;
156     }
157     synchronized (otherRegionServers) {
158       LOG.info("Current list of replicators: " + currentReplicators
159           + " other RSs: " + otherRegionServers);
160     }
161     // Look if there's anything to process after a restart
162     for (String rs : currentReplicators) {
163       synchronized (otherRegionServers) {
164         if (!this.otherRegionServers.contains(rs)) {
165           transferQueues(rs);
166         }
167       }
168     }
169   }
170 
171   /**
172    * Add a new normal source to this region server
173    * @param id the id of the peer cluster
174    * @return the source that was created
175    * @throws IOException
176    */
177   public ReplicationSourceInterface addSource(String id) throws IOException {
178     ReplicationSourceInterface src =
179         getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
180     // TODO set it to what's in ZK
181     src.setSourceEnabled(true);
182     synchronized (this.hlogs) {
183       this.sources.add(src);
184       if (this.hlogs.size() > 0) {
185         // Add the latest hlog to that source's queue
186         this.zkHelper.addLogToList(this.hlogs.last(),
187             this.sources.get(0).getPeerClusterZnode());
188         src.enqueueLog(this.latestPath);
189       }
190     }
191     src.startup();
192     return src;
193   }
194 
195   /**
196    * Terminate the replication on this region server
197    */
198   public void join() {
199     if (this.sources.size() == 0) {
200       this.zkHelper.deleteOwnRSZNode();
201     }
202     for (ReplicationSourceInterface source : this.sources) {
203       source.terminate("Region server is closing");
204     }
205   }
206 
207   /**
208    * Get a copy of the hlogs of the first source on this rs
209    * @return a sorted set of hlog names
210    */
211   protected SortedSet<String> getHLogs() {
212     return new TreeSet<String>(this.hlogs);
213   }
214 
215   /**
216    * Get a list of all the normal sources of this rs
217    * @return lis of all sources
218    */
219   public List<ReplicationSourceInterface> getSources() {
220     return this.sources;
221   }
222 
223   void logRolled(Path newLog) {
224     if (!this.replicating.get()) {
225       LOG.warn("Replication stopped, won't add new log");
226       return;
227     }
228     
229     if (this.sources.size() > 0) {
230       this.zkHelper.addLogToList(newLog.getName(),
231           this.sources.get(0).getPeerClusterZnode());
232     }
233     synchronized (this.hlogs) {
234       this.hlogs.add(newLog.getName());
235     }
236     this.latestPath = newLog;
237     // This only update the sources we own, not the recovered ones
238     for (ReplicationSourceInterface source : this.sources) {
239       source.enqueueLog(newLog);
240     }
241   }
242 
243   /**
244    * Get the ZK help of this manager
245    * @return the helper
246    */
247   public ReplicationZookeeper getRepZkWrapper() {
248     return zkHelper;
249   }
250 
251   /**
252    * Factory method to create a replication source
253    * @param conf the configuration to use
254    * @param fs the file system to use
255    * @param manager the manager to use
256    * @param stopper the stopper object for this region server
257    * @param replicating the status of the replication on this cluster
258    * @param peerClusterId the id of the peer cluster
259    * @return the created source
260    * @throws IOException
261    */
262   public ReplicationSourceInterface getReplicationSource(
263       final Configuration conf,
264       final FileSystem fs,
265       final ReplicationSourceManager manager,
266       final Stoppable stopper,
267       final AtomicBoolean replicating,
268       final String peerClusterId) throws IOException {
269     ReplicationSourceInterface src;
270     try {
271       @SuppressWarnings("rawtypes")
272       Class c = Class.forName(conf.get("replication.replicationsource.implementation",
273           ReplicationSource.class.getCanonicalName()));
274       src = (ReplicationSourceInterface) c.newInstance();
275     } catch (Exception e) {
276       LOG.warn("Passed replication source implemention throws errors, " +
277           "defaulting to ReplicationSource", e);
278       src = new ReplicationSource();
279 
280     }
281     src.init(conf, fs, manager, stopper, replicating, peerClusterId);
282     return src;
283   }
284 
285   /**
286    * Transfer all the queues of the specified to this region server.
287    * First it tries to grab a lock and if it works it will move the
288    * znodes and finally will delete the old znodes.
289    *
290    * It creates one old source for any type of source of the old rs.
291    * @param rsZnode
292    */
293   public void transferQueues(String rsZnode) {
294     // We try to lock that rs' queue directory
295     if (this.stopper.isStopped()) {
296       LOG.info("Not transferring queue since we are shutting down");
297       return;
298     }
299     if (!this.zkHelper.lockOtherRS(rsZnode)) {
300       return;
301     }
302     LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
303     SortedMap<String, SortedSet<String>> newQueues =
304         this.zkHelper.copyQueuesFromRS(rsZnode);
305     this.zkHelper.deleteRsQueues(rsZnode);
306     if (newQueues == null || newQueues.size() == 0) {
307       return;
308     }
309 
310     for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
311       String peerId = entry.getKey();
312       try {
313         ReplicationSourceInterface src = getReplicationSource(this.conf,
314             this.fs, this, this.stopper, this.replicating, peerId);
315         if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
316           src.terminate("Recovered queue doesn't belong to any current peer");
317           break;
318         }
319         this.oldsources.add(src);
320         for (String hlog : entry.getValue()) {
321           src.enqueueLog(new Path(this.oldLogDir, hlog));
322         }
323         // TODO set it to what's in ZK
324         src.setSourceEnabled(true);
325         src.startup();
326       } catch (IOException e) {
327         // TODO manage it
328         LOG.error("Failed creating a source", e);
329       }
330     }
331   }
332 
333   /**
334    * Clear the references to the specified old source
335    * @param src source to clear
336    */
337   public void closeRecoveredQueue(ReplicationSourceInterface src) {
338     LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
339     this.oldsources.remove(src);
340     this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
341   }
342 
343   /**
344    * Thie method first deletes all the recovered sources for the specified
345    * id, then deletes the normal source (deleting all related data in ZK).
346    * @param id The id of the peer cluster
347    */
348   public void removePeer(String id) {
349     LOG.info("Closing the following queue " + id + ", currently have "
350         + sources.size() + " and another "
351         + oldsources.size() + " that were recovered");
352     ReplicationSourceInterface srcToRemove = null;
353     List<ReplicationSourceInterface> oldSourcesToDelete =
354         new ArrayList<ReplicationSourceInterface>();
355     // First close all the recovered sources for this peer
356     for (ReplicationSourceInterface src : oldsources) {
357       if (id.equals(src.getPeerClusterId())) {
358         oldSourcesToDelete.add(src);
359       }
360     }
361     for (ReplicationSourceInterface src : oldSourcesToDelete) {
362       closeRecoveredQueue((src));
363     }
364     LOG.info("Number of deleted recovered sources for " + id + ": "
365         + oldSourcesToDelete.size());
366     // Now look for the one on this cluster
367     for (ReplicationSourceInterface src : this.sources) {
368       if (id.equals(src.getPeerClusterId())) {
369         srcToRemove = src;
370         break;
371       }
372     }
373     if (srcToRemove == null) {
374       LOG.error("The queue we wanted to close is missing " + id);
375       return;
376     }
377     srcToRemove.terminate("Replication stream was removed by a user");
378     this.sources.remove(srcToRemove);
379     this.zkHelper.deleteSource(id, true);
380   }
381 
382   /**
383    * Watcher used to be notified of the other region server's death
384    * in the local cluster. It initiates the process to transfer the queues
385    * if it is able to grab the lock.
386    */
387   public class OtherRegionServerWatcher extends ZooKeeperListener {
388 
389     /**
390      * Construct a ZooKeeper event listener.
391      */
392     public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
393       super(watcher);
394     }
395 
396     /**
397      * Called when a new node has been created.
398      * @param path full path of the new node
399      */
400     public void nodeCreated(String path) {
401       refreshRegionServersList(path);
402     }
403 
404     /**
405      * Called when a node has been deleted
406      * @param path full path of the deleted node
407      */
408     public void nodeDeleted(String path) {
409       if (stopper.isStopped()) {
410         return;
411       }
412       boolean cont = refreshRegionServersList(path);
413       if (!cont) {
414         return;
415       }
416       LOG.info(path + " znode expired, trying to lock it");
417       transferQueues(zkHelper.getZNodeName(path));
418     }
419 
420     /**
421      * Called when an existing node has a child node added or removed.
422      * @param path full path of the node whose children have changed
423      */
424     public void nodeChildrenChanged(String path) {
425       if (stopper.isStopped()) {
426         return;
427       }
428       refreshRegionServersList(path);
429     }
430 
431     private boolean refreshRegionServersList(String path) {
432       if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
433         return false;
434       }
435       List<String> newRsList = (zkHelper.getRegisteredRegionServers());
436       if (newRsList == null) {
437         return false;
438       } else {
439         synchronized (otherRegionServers) {
440           otherRegionServers.clear();
441           otherRegionServers.addAll(newRsList);
442         }
443       }
444       return true;
445     }
446   }
447 
448   /**
449    * Watcher used to follow the creation and deletion of peer clusters.
450    */
451   public class PeersWatcher extends ZooKeeperListener {
452 
453     /**
454      * Construct a ZooKeeper event listener.
455      */
456     public PeersWatcher(ZooKeeperWatcher watcher) {
457       super(watcher);
458     }
459 
460     /**
461      * Called when a node has been deleted
462      * @param path full path of the deleted node
463      */
464     public void nodeDeleted(String path) {
465       List<String> peers = refreshPeersList(path);
466       if (peers == null) {
467         return;
468       }
469       String id = zkHelper.getZNodeName(path);
470       removePeer(id);
471     }
472 
473     /**
474      * Called when an existing node has a child node added or removed.
475      * @param path full path of the node whose children have changed
476      */
477     public void nodeChildrenChanged(String path) {
478       List<String> peers = refreshPeersList(path);
479       if (peers == null) {
480         return;
481       }
482       for (String id : peers) {
483         try {
484           boolean added = zkHelper.connectToPeer(id);
485           if (added) {
486             addSource(id);
487           }
488         } catch (IOException e) {
489           // TODO manage better than that ?
490           LOG.error("Error while adding a new peer", e);
491         } catch (KeeperException e) {
492           LOG.error("Error while adding a new peer", e);
493         }
494       }
495     }
496 
497     /**
498      * Verify if this event is meant for us, and if so then get the latest
499      * peers' list from ZK. Also reset the watches.
500      * @param path path to check against
501      * @return A list of peers' identifiers if the event concerns this watcher,
502      * else null.
503      */
504     private List<String> refreshPeersList(String path) {
505       if (!path.startsWith(zkHelper.getPeersZNode())) {
506         return null;
507       }
508       return zkHelper.listPeersIdsAndWatch();
509     }
510   }
511 
512   /**
513    * Get the directory where hlogs are archived
514    * @return the directory where hlogs are archived
515    */
516   public Path getOldLogDir() {
517     return this.oldLogDir;
518   }
519 
520   /**
521    * Get the directory where hlogs are stored by their RSs
522    * @return the directory where hlogs are stored by their RSs
523    */
524   public Path getLogDir() {
525     return this.logDir;
526   }
527 
528   /**
529    * Get the handle on the local file system
530    * @return Handle on the local file system
531    */
532   public FileSystem getFs() {
533     return this.fs;
534   }
535 }