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.Collections;
26  import java.util.HashMap;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Random;
30  import java.util.SortedMap;
31  import java.util.SortedSet;
32  import java.util.TreeSet;
33  import java.util.concurrent.LinkedBlockingQueue;
34  import java.util.concurrent.RejectedExecutionException;
35  import java.util.concurrent.ThreadPoolExecutor;
36  import java.util.concurrent.TimeUnit;
37  import java.util.concurrent.atomic.AtomicBoolean;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FileSystem;
43  import org.apache.hadoop.fs.Path;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.Stoppable;
46  import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
47  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
48  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
49  import org.apache.zookeeper.KeeperException;
50  
51  import com.google.common.util.concurrent.ThreadFactoryBuilder;
52  
53  /**
54   * This class is responsible to manage all the replication
55   * sources. There are two classes of sources:
56   * <li> Normal sources are persistent and one per peer cluster</li>
57   * <li> Old sources are recovered from a failed region server and our
58   * only goal is to finish replicating the HLog queue it had up in ZK</li>
59   *
60   * When a region server dies, this class uses a watcher to get notified and it
61   * tries to grab a lock in order to transfer all the queues in a local
62   * old source.
63   */
64  public class ReplicationSourceManager {
65    private static final Log LOG =
66        LogFactory.getLog(ReplicationSourceManager.class);
67    // List of all the sources that read this RS's logs
68    private final List<ReplicationSourceInterface> sources;
69    // List of all the sources we got from died RSs
70    private final List<ReplicationSourceInterface> oldsources;
71    // Indicates if we are currently replicating
72    private final AtomicBoolean replicating;
73    // Helper for zookeeper
74    private final ReplicationZookeeper zkHelper;
75    // All about stopping
76    private final Stoppable stopper;
77    // All logs we are currently trackign
78    private final Map<String, SortedSet<String>> hlogsById;
79    private final Configuration conf;
80    private final FileSystem fs;
81    // The path to the latest log we saw, for new coming sources
82    private Path latestPath;
83    // List of all the other region servers in this cluster
84    private final List<String> otherRegionServers = new ArrayList<String>();
85    // Path to the hlogs directories
86    private final Path logDir;
87    // Path to the hlog archive
88    private final Path oldLogDir;
89    // The number of ms that we wait before moving znodes, HBASE-3596
90    private final long sleepBeforeFailover;
91    // Homemade executer service for replication
92    private final ThreadPoolExecutor executor;
93    
94    private final Random rand;
95  
96  
97    /**
98     * Creates a replication manager and sets the watch on all the other
99     * registered region servers
100    * @param zkHelper the zk helper for replication
101    * @param conf the configuration to use
102    * @param stopper the stopper object for this region server
103    * @param fs the file system to use
104    * @param replicating the status of the replication on this cluster
105    * @param logDir the directory that contains all hlog directories of live RSs
106    * @param oldLogDir the directory where old logs are archived
107    */
108   public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
109                                   final Configuration conf,
110                                   final Stoppable stopper,
111                                   final FileSystem fs,
112                                   final AtomicBoolean replicating,
113                                   final Path logDir,
114                                   final Path oldLogDir) {
115     this.sources = new ArrayList<ReplicationSourceInterface>();
116     this.replicating = replicating;
117     this.zkHelper = zkHelper;
118     this.stopper = stopper;
119     this.hlogsById = new HashMap<String, SortedSet<String>>();
120     this.oldsources = new ArrayList<ReplicationSourceInterface>();
121     this.conf = conf;
122     this.fs = fs;
123     this.logDir = logDir;
124     this.oldLogDir = oldLogDir;
125     this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
126     this.zkHelper.registerRegionServerListener(
127         new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
128     this.zkHelper.registerRegionServerListener(
129         new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
130     this.zkHelper.listPeersIdsAndWatch();
131     // It's preferable to failover 1 RS at a time, but with good zk servers
132     // more could be processed at the same time.
133     int nbWorkers = conf.getInt("replication.executor.workers", 1);
134     // use a short 100ms sleep since this could be done inline with a RS startup
135     // even if we fail, other region servers can take care of it
136     this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
137         100, TimeUnit.MILLISECONDS,
138         new LinkedBlockingQueue<Runnable>());
139     ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
140     tfb.setNameFormat("ReplicationExecutor-%d");
141     this.executor.setThreadFactory(tfb.build());
142     this.rand = new Random();
143   }
144 
145   /**
146    * Provide the id of the peer and a log key and this method will figure which
147    * hlog it belongs to and will log, for this region server, the current
148    * position. It will also clean old logs from the queue.
149    * @param log Path to the log currently being replicated from
150    * replication status in zookeeper. It will also delete older entries.
151    * @param id id of the peer cluster
152    * @param position current location in the log
153    * @param queueRecovered indicates if this queue comes from another region server
154    * @param holdLogInZK if true then the log is retained in ZK
155    */
156   public void logPositionAndCleanOldLogs(Path log, String id, long position, 
157       boolean queueRecovered, boolean holdLogInZK) {
158     String key = log.getName();
159     LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
160     this.zkHelper.writeReplicationStatus(key, id, position);
161     if (holdLogInZK) {
162      return;
163     }
164     cleanOldLogs(key, id, queueRecovered);
165   }
166 
167   /**
168    * Cleans a log file and all older files from ZK. Called when we are sure that a
169    * log file is closed and has no more entries.
170    * @param key Path to the log
171    * @param id id of the peer cluster
172    * @param queueRecovered Whether this is a recovered queue
173    */
174   public void cleanOldLogs(String key,
175                            String id,
176                            boolean queueRecovered) {
177     synchronized (this.hlogsById) {
178       SortedSet<String> hlogs = this.hlogsById.get(id);
179       if (queueRecovered || hlogs.first().equals(key)) {
180         return;
181       }
182       SortedSet<String> hlogSet = hlogs.headSet(key);
183       for (String hlog : hlogSet) {
184         this.zkHelper.removeLogFromList(hlog, id);
185       }
186       hlogSet.clear();
187     }
188   }
189 
190   /**
191    * Adds a normal source per registered peer cluster and tries to process all
192    * old region server hlog queues
193    */
194   public void init() throws IOException {
195     for (String id : this.zkHelper.getPeerClusters().keySet()) {
196       addSource(id);
197     }
198     List<String> currentReplicators = this.zkHelper.getListOfReplicators();
199     if (currentReplicators == null || currentReplicators.size() == 0) {
200       return;
201     }
202     synchronized (otherRegionServers) {
203       refreshOtherRegionServersList();
204       LOG.info("Current list of replicators: " + currentReplicators
205           + " other RSs: " + otherRegionServers);
206     }
207     // Look if there's anything to process after a restart
208     for (String rs : currentReplicators) {
209       synchronized (otherRegionServers) {
210         if (!this.otherRegionServers.contains(rs)) {
211           transferQueues(rs);
212         }
213       }
214     }
215   }
216 
217   /**
218    * Add a new normal source to this region server
219    * @param id the id of the peer cluster
220    * @return the source that was created
221    * @throws IOException
222    */
223   public ReplicationSourceInterface addSource(String id) throws IOException {
224     ReplicationSourceInterface src =
225         getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
226     synchronized (this.hlogsById) {
227       this.sources.add(src);
228       this.hlogsById.put(id, new TreeSet<String>());
229       // Add the latest hlog to that source's queue
230       if (this.latestPath != null) {
231         String name = this.latestPath.getName();
232         this.hlogsById.get(id).add(name);
233         try {
234           this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
235         } catch (KeeperException ke) {
236           String message = "Cannot add log to zk for" +
237             " replication when creating a new source";
238           stopper.stop(message);
239           throw new IOException(message, ke);
240         }
241         src.enqueueLog(this.latestPath);
242       }
243     }
244     src.startup();
245     return src;
246   }
247 
248   /**
249    * Terminate the replication on this region server
250    */
251   public void join() {
252     this.executor.shutdown();
253     if (this.sources.size() == 0) {
254       this.zkHelper.deleteOwnRSZNode();
255     }
256     for (ReplicationSourceInterface source : this.sources) {
257       source.terminate("Region server is closing");
258     }
259   }
260 
261   /**
262    * Get a copy of the hlogs of the first source on this rs
263    * @return a sorted set of hlog names
264    */
265   protected Map<String, SortedSet<String>> getHLogs() {
266     return Collections.unmodifiableMap(hlogsById);
267   }
268 
269   /**
270    * Get a list of all the normal sources of this rs
271    * @return lis of all sources
272    */
273   public List<ReplicationSourceInterface> getSources() {
274     return this.sources;
275   }
276 
277   void preLogRoll(Path newLog) throws IOException {
278     if (!this.replicating.get()) {
279       LOG.warn("Replication stopped, won't add new log");
280       return;
281     }
282 
283     synchronized (this.hlogsById) {
284       String name = newLog.getName();
285       for (ReplicationSourceInterface source : this.sources) {
286         try {
287           this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
288         } catch (KeeperException ke) {
289           throw new IOException("Cannot add log to zk for replication", ke);
290         }
291       }
292       for (SortedSet<String> hlogs : this.hlogsById.values()) {
293         if (this.sources.isEmpty()) {
294           // If there's no slaves, don't need to keep the old hlogs since
295           // we only consider the last one when a new slave comes in
296           hlogs.clear();
297         }
298         hlogs.add(name);
299       }
300     }
301 
302     this.latestPath = newLog;
303   }
304 
305   void postLogRoll(Path newLog) throws IOException {
306     if (!this.replicating.get()) {
307       LOG.warn("Replication stopped, won't add new log");
308       return;
309     }
310 
311     // This only updates the sources we own, not the recovered ones
312     for (ReplicationSourceInterface source : this.sources) {
313       source.enqueueLog(newLog);    
314     }
315   }
316 
317   /**
318    * Get the ZK help of this manager
319    * @return the helper
320    */
321   public ReplicationZookeeper getRepZkWrapper() {
322     return zkHelper;
323   }
324 
325   /**
326    * Factory method to create a replication source
327    * @param conf the configuration to use
328    * @param fs the file system to use
329    * @param manager the manager to use
330    * @param stopper the stopper object for this region server
331    * @param replicating the status of the replication on this cluster
332    * @param peerId the id of the peer cluster
333    * @return the created source
334    * @throws IOException
335    */
336   public ReplicationSourceInterface getReplicationSource(
337       final Configuration conf,
338       final FileSystem fs,
339       final ReplicationSourceManager manager,
340       final Stoppable stopper,
341       final AtomicBoolean replicating,
342       final String peerId) throws IOException {
343     ReplicationSourceInterface src;
344     try {
345       @SuppressWarnings("rawtypes")
346       Class c = Class.forName(conf.get("replication.replicationsource.implementation",
347           ReplicationSource.class.getCanonicalName()));
348       src = (ReplicationSourceInterface) c.newInstance();
349     } catch (Exception e) {
350       LOG.warn("Passed replication source implementation throws errors, " +
351           "defaulting to ReplicationSource", e);
352       src = new ReplicationSource();
353 
354     }
355     src.init(conf, fs, manager, stopper, replicating, peerId);
356     return src;
357   }
358 
359   /**
360    * Transfer all the queues of the specified to this region server.
361    * First it tries to grab a lock and if it works it will move the
362    * znodes and finally will delete the old znodes.
363    *
364    * It creates one old source for any type of source of the old rs.
365    * @param rsZnode
366    */
367   public void transferQueues(String rsZnode) {
368     NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
369     try {
370       this.executor.execute(transfer);
371     } catch (RejectedExecutionException ex) {
372       LOG.info("Cancelling the transfer of " + rsZnode +
373           " because of " + ex.getMessage());
374     }
375   }
376 
377   /**
378    * Clear the references to the specified old source
379    * @param src source to clear
380    */
381   public void closeRecoveredQueue(ReplicationSourceInterface src) {
382     LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
383     this.oldsources.remove(src);
384     this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
385   }
386 
387   /**
388    * Thie method first deletes all the recovered sources for the specified
389    * id, then deletes the normal source (deleting all related data in ZK).
390    * @param id The id of the peer cluster
391    */
392   public void removePeer(String id) {
393     LOG.info("Closing the following queue " + id + ", currently have "
394         + sources.size() + " and another "
395         + oldsources.size() + " that were recovered");
396     String terminateMessage = "Replication stream was removed by a user";
397     ReplicationSourceInterface srcToRemove = null;
398     List<ReplicationSourceInterface> oldSourcesToDelete =
399         new ArrayList<ReplicationSourceInterface>();
400     // First close all the recovered sources for this peer
401     for (ReplicationSourceInterface src : oldsources) {
402       if (id.equals(src.getPeerClusterId())) {
403         oldSourcesToDelete.add(src);
404       }
405     }
406     for (ReplicationSourceInterface src : oldSourcesToDelete) {
407       src.terminate(terminateMessage);
408       closeRecoveredQueue((src));
409     }
410     LOG.info("Number of deleted recovered sources for " + id + ": "
411         + oldSourcesToDelete.size());
412     // Now look for the one on this cluster
413     for (ReplicationSourceInterface src : this.sources) {
414       if (id.equals(src.getPeerClusterId())) {
415         srcToRemove = src;
416         break;
417       }
418     }
419     if (srcToRemove == null) {
420       LOG.error("The queue we wanted to close is missing " + id);
421       return;
422     }
423     srcToRemove.terminate(terminateMessage);
424     this.sources.remove(srcToRemove);
425     this.zkHelper.deleteSource(id, true);
426   }
427 
428   /**
429    * Reads the list of region servers from ZK and atomically clears our
430    * local view of it and replaces it with the updated list.
431    * 
432    * @return true if the local list of the other region servers was updated
433    * with the ZK data (even if it was empty),
434    * false if the data was missing in ZK
435    */
436   private boolean refreshOtherRegionServersList() {
437     List<String> newRsList = zkHelper.getRegisteredRegionServers();
438     if (newRsList == null) {
439       return false;
440     } else {
441       synchronized (otherRegionServers) {
442         otherRegionServers.clear();
443         otherRegionServers.addAll(newRsList);
444       }
445     }
446     return true;
447   }
448 
449   /**
450    * Watcher used to be notified of the other region server's death
451    * in the local cluster. It initiates the process to transfer the queues
452    * if it is able to grab the lock.
453    */
454   public class OtherRegionServerWatcher extends ZooKeeperListener {
455 
456     /**
457      * Construct a ZooKeeper event listener.
458      */
459     public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
460       super(watcher);
461     }
462 
463     /**
464      * Called when a new node has been created.
465      * @param path full path of the new node
466      */
467     public void nodeCreated(String path) {
468       refreshListIfRightPath(path);
469     }
470 
471     /**
472      * Called when a node has been deleted
473      * @param path full path of the deleted node
474      */
475     public void nodeDeleted(String path) {
476       if (stopper.isStopped()) {
477         return;
478       }
479       boolean cont = refreshListIfRightPath(path);
480       if (!cont) {
481         return;
482       }
483       LOG.info(path + " znode expired, trying to lock it");
484       transferQueues(ReplicationZookeeper.getZNodeName(path));
485     }
486 
487     /**
488      * Called when an existing node has a child node added or removed.
489      * @param path full path of the node whose children have changed
490      */
491     public void nodeChildrenChanged(String path) {
492       if (stopper.isStopped()) {
493         return;
494       }
495       refreshListIfRightPath(path);
496     }
497 
498     private boolean refreshListIfRightPath(String path) {
499       if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
500         return false;
501       }
502       return refreshOtherRegionServersList();
503     }
504   }
505 
506   /**
507    * Watcher used to follow the creation and deletion of peer clusters.
508    */
509   public class PeersWatcher extends ZooKeeperListener {
510 
511     /**
512      * Construct a ZooKeeper event listener.
513      */
514     public PeersWatcher(ZooKeeperWatcher watcher) {
515       super(watcher);
516     }
517 
518     /**
519      * Called when a node has been deleted
520      * @param path full path of the deleted node
521      */
522     public void nodeDeleted(String path) {
523       List<String> peers = refreshPeersList(path);
524       if (peers == null) {
525         return;
526       }
527       String id = ReplicationZookeeper.getZNodeName(path);
528       removePeer(id);
529     }
530 
531     /**
532      * Called when an existing node has a child node added or removed.
533      * @param path full path of the node whose children have changed
534      */
535     public void nodeChildrenChanged(String path) {
536       List<String> peers = refreshPeersList(path);
537       if (peers == null) {
538         return;
539       }
540       for (String id : peers) {
541         try {
542           boolean added = zkHelper.connectToPeer(id);
543           if (added) {
544             addSource(id);
545           }
546         } catch (IOException e) {
547           // TODO manage better than that ?
548           LOG.error("Error while adding a new peer", e);
549         } catch (KeeperException e) {
550           LOG.error("Error while adding a new peer", e);
551         }
552       }
553     }
554 
555     /**
556      * Verify if this event is meant for us, and if so then get the latest
557      * peers' list from ZK. Also reset the watches.
558      * @param path path to check against
559      * @return A list of peers' identifiers if the event concerns this watcher,
560      * else null.
561      */
562     private List<String> refreshPeersList(String path) {
563       if (!path.startsWith(zkHelper.getPeersZNode())) {
564         return null;
565       }
566       return zkHelper.listPeersIdsAndWatch();
567     }
568   }
569 
570   /**
571    * Class responsible to setup new ReplicationSources to take care of the
572    * queues from dead region servers.
573    */
574   class NodeFailoverWorker extends Thread {
575 
576     private String rsZnode;
577 
578     /**
579      *
580      * @param rsZnode
581      */
582     public NodeFailoverWorker(String rsZnode) {
583       super("Failover-for-"+rsZnode);
584       this.rsZnode = rsZnode;
585     }
586 
587     @Override
588     public void run() {
589       // Wait a bit before transferring the queues, we may be shutting down.
590       // This sleep may not be enough in some cases.
591       try {
592         Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
593       } catch (InterruptedException e) {
594         LOG.warn("Interrupted while waiting before transferring a queue.");
595         Thread.currentThread().interrupt();
596       }
597       // We try to lock that rs' queue directory
598       if (stopper.isStopped()) {
599         LOG.info("Not transferring queue since we are shutting down");
600         return;
601       }
602       SortedMap<String, SortedSet<String>> newQueues = null;
603 
604       // check whether there is multi support. If yes, use it.
605       if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
606         LOG.info("Atomically moving " + rsZnode + "'s hlogs to my queue");
607         newQueues = zkHelper.copyQueuesFromRSUsingMulti(rsZnode);
608       } else {
609         LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
610         if (!zkHelper.lockOtherRS(rsZnode)) {
611           return;
612         }
613         newQueues = zkHelper.copyQueuesFromRS(rsZnode);
614         zkHelper.deleteRsQueues(rsZnode);
615       }
616       // process of copying over the failed queue is completed.
617       if (newQueues.isEmpty()) {
618         return;
619       }
620 
621       for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
622         String peerId = entry.getKey();
623         try {
624           ReplicationSourceInterface src = getReplicationSource(conf,
625               fs, ReplicationSourceManager.this, stopper, replicating, peerId);
626           if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
627             src.terminate("Recovered queue doesn't belong to any current peer");
628             break;
629           }
630           oldsources.add(src);
631           for (String hlog : entry.getValue()) {
632             src.enqueueLog(new Path(oldLogDir, hlog));
633           }
634           src.startup();
635         } catch (IOException e) {
636           // TODO manage it
637           LOG.error("Failed creating a source", e);
638         }
639       }
640     }
641   }
642 
643   /**
644    * Get the directory where hlogs are archived
645    * @return the directory where hlogs are archived
646    */
647   public Path getOldLogDir() {
648     return this.oldLogDir;
649   }
650 
651   /**
652    * Get the directory where hlogs are stored by their RSs
653    * @return the directory where hlogs are stored by their RSs
654    */
655   public Path getLogDir() {
656     return this.logDir;
657   }
658 
659   /**
660    * Get the handle on the local file system
661    * @return Handle on the local file system
662    */
663   public FileSystem getFs() {
664     return this.fs;
665   }
666 }