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     synchronized (this.hlogsById) {
165       SortedSet<String> hlogs = this.hlogsById.get(id);
166       if (!queueRecovered && hlogs.first() != key) {
167         SortedSet<String> hlogSet = hlogs.headSet(key);
168         LOG.info("Removing " + hlogSet.size() +
169             " logs in the list: " + hlogSet);
170         for (String hlog : hlogSet) {
171           this.zkHelper.removeLogFromList(hlog, id);
172         }
173         hlogSet.clear();
174       }
175     }
176   }
177 
178   /**
179    * Adds a normal source per registered peer cluster and tries to process all
180    * old region server hlog queues
181    */
182   public void init() throws IOException {
183     for (String id : this.zkHelper.getPeerClusters().keySet()) {
184       addSource(id);
185     }
186     List<String> currentReplicators = this.zkHelper.getListOfReplicators();
187     if (currentReplicators == null || currentReplicators.size() == 0) {
188       return;
189     }
190     synchronized (otherRegionServers) {
191       refreshOtherRegionServersList();
192       LOG.info("Current list of replicators: " + currentReplicators
193           + " other RSs: " + otherRegionServers);
194     }
195     // Look if there's anything to process after a restart
196     for (String rs : currentReplicators) {
197       synchronized (otherRegionServers) {
198         if (!this.otherRegionServers.contains(rs)) {
199           transferQueues(rs);
200         }
201       }
202     }
203   }
204 
205   /**
206    * Add a new normal source to this region server
207    * @param id the id of the peer cluster
208    * @return the source that was created
209    * @throws IOException
210    */
211   public ReplicationSourceInterface addSource(String id) throws IOException {
212     ReplicationSourceInterface src =
213         getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
214     synchronized (this.hlogsById) {
215       this.sources.add(src);
216       this.hlogsById.put(id, new TreeSet<String>());
217       // Add the latest hlog to that source's queue
218       if (this.latestPath != null) {
219         String name = this.latestPath.getName();
220         this.hlogsById.get(id).add(name);
221         try {
222           this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
223         } catch (KeeperException ke) {
224           String message = "Cannot add log to zk for" +
225             " replication when creating a new source";
226           stopper.stop(message);
227           throw new IOException(message, ke);
228         }
229         src.enqueueLog(this.latestPath);
230       }
231     }
232     src.startup();
233     return src;
234   }
235 
236   /**
237    * Terminate the replication on this region server
238    */
239   public void join() {
240     this.executor.shutdown();
241     if (this.sources.size() == 0) {
242       this.zkHelper.deleteOwnRSZNode();
243     }
244     for (ReplicationSourceInterface source : this.sources) {
245       source.terminate("Region server is closing");
246     }
247   }
248 
249   /**
250    * Get a copy of the hlogs of the first source on this rs
251    * @return a sorted set of hlog names
252    */
253   protected Map<String, SortedSet<String>> getHLogs() {
254     return Collections.unmodifiableMap(hlogsById);
255   }
256 
257   /**
258    * Get a list of all the normal sources of this rs
259    * @return lis of all sources
260    */
261   public List<ReplicationSourceInterface> getSources() {
262     return this.sources;
263   }
264 
265   void preLogRoll(Path newLog) throws IOException {
266     if (!this.replicating.get()) {
267       LOG.warn("Replication stopped, won't add new log");
268       return;
269     }
270 
271     synchronized (this.hlogsById) {
272       String name = newLog.getName();
273       for (ReplicationSourceInterface source : this.sources) {
274         try {
275           this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
276         } catch (KeeperException ke) {
277           throw new IOException("Cannot add log to zk for replication", ke);
278         }
279       }
280       for (SortedSet<String> hlogs : this.hlogsById.values()) {
281         if (this.sources.isEmpty()) {
282           // If there's no slaves, don't need to keep the old hlogs since
283           // we only consider the last one when a new slave comes in
284           hlogs.clear();
285         }
286         hlogs.add(name);
287       }
288     }
289 
290     this.latestPath = newLog;
291   }
292 
293   void postLogRoll(Path newLog) throws IOException {
294     if (!this.replicating.get()) {
295       LOG.warn("Replication stopped, won't add new log");
296       return;
297     }
298 
299     // This only updates the sources we own, not the recovered ones
300     for (ReplicationSourceInterface source : this.sources) {
301       source.enqueueLog(newLog);    
302     }
303   }
304 
305   /**
306    * Get the ZK help of this manager
307    * @return the helper
308    */
309   public ReplicationZookeeper getRepZkWrapper() {
310     return zkHelper;
311   }
312 
313   /**
314    * Factory method to create a replication source
315    * @param conf the configuration to use
316    * @param fs the file system to use
317    * @param manager the manager to use
318    * @param stopper the stopper object for this region server
319    * @param replicating the status of the replication on this cluster
320    * @param peerId the id of the peer cluster
321    * @return the created source
322    * @throws IOException
323    */
324   public ReplicationSourceInterface getReplicationSource(
325       final Configuration conf,
326       final FileSystem fs,
327       final ReplicationSourceManager manager,
328       final Stoppable stopper,
329       final AtomicBoolean replicating,
330       final String peerId) throws IOException {
331     ReplicationSourceInterface src;
332     try {
333       @SuppressWarnings("rawtypes")
334       Class c = Class.forName(conf.get("replication.replicationsource.implementation",
335           ReplicationSource.class.getCanonicalName()));
336       src = (ReplicationSourceInterface) c.newInstance();
337     } catch (Exception e) {
338       LOG.warn("Passed replication source implementation throws errors, " +
339           "defaulting to ReplicationSource", e);
340       src = new ReplicationSource();
341 
342     }
343     src.init(conf, fs, manager, stopper, replicating, peerId);
344     return src;
345   }
346 
347   /**
348    * Transfer all the queues of the specified to this region server.
349    * First it tries to grab a lock and if it works it will move the
350    * znodes and finally will delete the old znodes.
351    *
352    * It creates one old source for any type of source of the old rs.
353    * @param rsZnode
354    */
355   public void transferQueues(String rsZnode) {
356     NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
357     try {
358       this.executor.execute(transfer);
359     } catch (RejectedExecutionException ex) {
360       LOG.info("Cancelling the transfer of " + rsZnode +
361           " because of " + ex.getMessage());
362     }
363   }
364 
365   /**
366    * Clear the references to the specified old source
367    * @param src source to clear
368    */
369   public void closeRecoveredQueue(ReplicationSourceInterface src) {
370     LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
371     this.oldsources.remove(src);
372     this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
373   }
374 
375   /**
376    * Thie method first deletes all the recovered sources for the specified
377    * id, then deletes the normal source (deleting all related data in ZK).
378    * @param id The id of the peer cluster
379    */
380   public void removePeer(String id) {
381     LOG.info("Closing the following queue " + id + ", currently have "
382         + sources.size() + " and another "
383         + oldsources.size() + " that were recovered");
384     String terminateMessage = "Replication stream was removed by a user";
385     ReplicationSourceInterface srcToRemove = null;
386     List<ReplicationSourceInterface> oldSourcesToDelete =
387         new ArrayList<ReplicationSourceInterface>();
388     // First close all the recovered sources for this peer
389     for (ReplicationSourceInterface src : oldsources) {
390       if (id.equals(src.getPeerClusterId())) {
391         oldSourcesToDelete.add(src);
392       }
393     }
394     for (ReplicationSourceInterface src : oldSourcesToDelete) {
395       src.terminate(terminateMessage);
396       closeRecoveredQueue((src));
397     }
398     LOG.info("Number of deleted recovered sources for " + id + ": "
399         + oldSourcesToDelete.size());
400     // Now look for the one on this cluster
401     for (ReplicationSourceInterface src : this.sources) {
402       if (id.equals(src.getPeerClusterId())) {
403         srcToRemove = src;
404         break;
405       }
406     }
407     if (srcToRemove == null) {
408       LOG.error("The queue we wanted to close is missing " + id);
409       return;
410     }
411     srcToRemove.terminate(terminateMessage);
412     this.sources.remove(srcToRemove);
413     this.zkHelper.deleteSource(id, true);
414   }
415 
416   /**
417    * Reads the list of region servers from ZK and atomically clears our
418    * local view of it and replaces it with the updated list.
419    * 
420    * @return true if the local list of the other region servers was updated
421    * with the ZK data (even if it was empty),
422    * false if the data was missing in ZK
423    */
424   private boolean refreshOtherRegionServersList() {
425     List<String> newRsList = zkHelper.getRegisteredRegionServers();
426     if (newRsList == null) {
427       return false;
428     } else {
429       synchronized (otherRegionServers) {
430         otherRegionServers.clear();
431         otherRegionServers.addAll(newRsList);
432       }
433     }
434     return true;
435   }
436 
437   /**
438    * Watcher used to be notified of the other region server's death
439    * in the local cluster. It initiates the process to transfer the queues
440    * if it is able to grab the lock.
441    */
442   public class OtherRegionServerWatcher extends ZooKeeperListener {
443 
444     /**
445      * Construct a ZooKeeper event listener.
446      */
447     public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
448       super(watcher);
449     }
450 
451     /**
452      * Called when a new node has been created.
453      * @param path full path of the new node
454      */
455     public void nodeCreated(String path) {
456       refreshListIfRightPath(path);
457     }
458 
459     /**
460      * Called when a node has been deleted
461      * @param path full path of the deleted node
462      */
463     public void nodeDeleted(String path) {
464       if (stopper.isStopped()) {
465         return;
466       }
467       boolean cont = refreshListIfRightPath(path);
468       if (!cont) {
469         return;
470       }
471       LOG.info(path + " znode expired, trying to lock it");
472       transferQueues(ReplicationZookeeper.getZNodeName(path));
473     }
474 
475     /**
476      * Called when an existing node has a child node added or removed.
477      * @param path full path of the node whose children have changed
478      */
479     public void nodeChildrenChanged(String path) {
480       if (stopper.isStopped()) {
481         return;
482       }
483       refreshListIfRightPath(path);
484     }
485 
486     private boolean refreshListIfRightPath(String path) {
487       if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
488         return false;
489       }
490       return refreshOtherRegionServersList();
491     }
492   }
493 
494   /**
495    * Watcher used to follow the creation and deletion of peer clusters.
496    */
497   public class PeersWatcher extends ZooKeeperListener {
498 
499     /**
500      * Construct a ZooKeeper event listener.
501      */
502     public PeersWatcher(ZooKeeperWatcher watcher) {
503       super(watcher);
504     }
505 
506     /**
507      * Called when a node has been deleted
508      * @param path full path of the deleted node
509      */
510     public void nodeDeleted(String path) {
511       List<String> peers = refreshPeersList(path);
512       if (peers == null) {
513         return;
514       }
515       String id = ReplicationZookeeper.getZNodeName(path);
516       removePeer(id);
517     }
518 
519     /**
520      * Called when an existing node has a child node added or removed.
521      * @param path full path of the node whose children have changed
522      */
523     public void nodeChildrenChanged(String path) {
524       List<String> peers = refreshPeersList(path);
525       if (peers == null) {
526         return;
527       }
528       for (String id : peers) {
529         try {
530           boolean added = zkHelper.connectToPeer(id);
531           if (added) {
532             addSource(id);
533           }
534         } catch (IOException e) {
535           // TODO manage better than that ?
536           LOG.error("Error while adding a new peer", e);
537         } catch (KeeperException e) {
538           LOG.error("Error while adding a new peer", e);
539         }
540       }
541     }
542 
543     /**
544      * Verify if this event is meant for us, and if so then get the latest
545      * peers' list from ZK. Also reset the watches.
546      * @param path path to check against
547      * @return A list of peers' identifiers if the event concerns this watcher,
548      * else null.
549      */
550     private List<String> refreshPeersList(String path) {
551       if (!path.startsWith(zkHelper.getPeersZNode())) {
552         return null;
553       }
554       return zkHelper.listPeersIdsAndWatch();
555     }
556   }
557 
558   /**
559    * Class responsible to setup new ReplicationSources to take care of the
560    * queues from dead region servers.
561    */
562   class NodeFailoverWorker extends Thread {
563 
564     private String rsZnode;
565 
566     /**
567      *
568      * @param rsZnode
569      */
570     public NodeFailoverWorker(String rsZnode) {
571       super("Failover-for-"+rsZnode);
572       this.rsZnode = rsZnode;
573     }
574 
575     @Override
576     public void run() {
577       // Wait a bit before transferring the queues, we may be shutting down.
578       // This sleep may not be enough in some cases.
579       try {
580         Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
581       } catch (InterruptedException e) {
582         LOG.warn("Interrupted while waiting before transferring a queue.");
583         Thread.currentThread().interrupt();
584       }
585       // We try to lock that rs' queue directory
586       if (stopper.isStopped()) {
587         LOG.info("Not transferring queue since we are shutting down");
588         return;
589       }
590       SortedMap<String, SortedSet<String>> newQueues = null;
591 
592       // check whether there is multi support. If yes, use it.
593       if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
594         LOG.info("Atomically moving " + rsZnode + "'s hlogs to my queue");
595         newQueues = zkHelper.copyQueuesFromRSUsingMulti(rsZnode);
596       } else {
597         LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
598         if (!zkHelper.lockOtherRS(rsZnode)) {
599           return;
600         }
601         newQueues = zkHelper.copyQueuesFromRS(rsZnode);
602         zkHelper.deleteRsQueues(rsZnode);
603       }
604       // process of copying over the failed queue is completed.
605       if (newQueues.isEmpty()) {
606         return;
607       }
608 
609       for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
610         String peerId = entry.getKey();
611         try {
612           ReplicationSourceInterface src = getReplicationSource(conf,
613               fs, ReplicationSourceManager.this, stopper, replicating, peerId);
614           if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
615             src.terminate("Recovered queue doesn't belong to any current peer");
616             break;
617           }
618           oldsources.add(src);
619           for (String hlog : entry.getValue()) {
620             src.enqueueLog(new Path(oldLogDir, hlog));
621           }
622           src.startup();
623         } catch (IOException e) {
624           // TODO manage it
625           LOG.error("Failed creating a source", e);
626         }
627       }
628     }
629   }
630 
631   /**
632    * Get the directory where hlogs are archived
633    * @return the directory where hlogs are archived
634    */
635   public Path getOldLogDir() {
636     return this.oldLogDir;
637   }
638 
639   /**
640    * Get the directory where hlogs are stored by their RSs
641    * @return the directory where hlogs are stored by their RSs
642    */
643   public Path getLogDir() {
644     return this.logDir;
645   }
646 
647   /**
648    * Get the handle on the local file system
649    * @return Handle on the local file system
650    */
651   public FileSystem getFs() {
652     return this.fs;
653   }
654 }