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