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