1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.replication.regionserver;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collections;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Random;
29 import java.util.SortedMap;
30 import java.util.SortedSet;
31 import java.util.TreeSet;
32 import java.util.UUID;
33 import java.util.concurrent.ConcurrentHashMap;
34 import java.util.concurrent.CopyOnWriteArrayList;
35 import java.util.concurrent.LinkedBlockingQueue;
36 import java.util.concurrent.RejectedExecutionException;
37 import java.util.concurrent.ThreadPoolExecutor;
38 import java.util.concurrent.TimeUnit;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.hbase.classification.InterfaceAudience;
43 import org.apache.hadoop.conf.Configuration;
44 import org.apache.hadoop.fs.FileSystem;
45 import org.apache.hadoop.fs.Path;
46 import org.apache.hadoop.hbase.Stoppable;
47 import org.apache.hadoop.hbase.replication.ReplicationException;
48 import org.apache.hadoop.hbase.replication.ReplicationListener;
49 import org.apache.hadoop.hbase.replication.ReplicationPeers;
50 import org.apache.hadoop.hbase.replication.ReplicationQueues;
51 import org.apache.hadoop.hbase.replication.ReplicationTracker;
52 import org.apache.zookeeper.KeeperException;
53
54 import com.google.common.util.concurrent.ThreadFactoryBuilder;
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70 @InterfaceAudience.Private
71 public class ReplicationSourceManager implements ReplicationListener {
72 private static final Log LOG =
73 LogFactory.getLog(ReplicationSourceManager.class);
74
75 private final List<ReplicationSourceInterface> sources;
76
77 private final List<ReplicationSourceInterface> oldsources;
78 private final ReplicationQueues replicationQueues;
79 private final ReplicationTracker replicationTracker;
80 private final ReplicationPeers replicationPeers;
81
82 private final UUID clusterId;
83
84 private final Stoppable stopper;
85
86 private final Map<String, SortedSet<String>> hlogsById;
87
88 private final Map<String, SortedSet<String>> hlogsByIdRecoveredQueues;
89 private final Configuration conf;
90 private final FileSystem fs;
91
92 private Path latestPath;
93
94 private final Path logDir;
95
96 private final Path oldLogDir;
97
98 private final long sleepBeforeFailover;
99
100 private final ThreadPoolExecutor executor;
101
102 private final Random rand;
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117 public ReplicationSourceManager(final ReplicationQueues replicationQueues,
118 final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
119 final Configuration conf, final Stoppable stopper, final FileSystem fs, final Path logDir,
120 final Path oldLogDir, final UUID clusterId) {
121
122
123 this.sources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
124 this.replicationQueues = replicationQueues;
125 this.replicationPeers = replicationPeers;
126 this.replicationTracker = replicationTracker;
127 this.stopper = stopper;
128 this.hlogsById = new HashMap<String, SortedSet<String>>();
129 this.hlogsByIdRecoveredQueues = new ConcurrentHashMap<String, SortedSet<String>>();
130 this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
131 this.conf = conf;
132 this.fs = fs;
133 this.logDir = logDir;
134 this.oldLogDir = oldLogDir;
135 this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
136 this.clusterId = clusterId;
137 this.replicationTracker.registerListener(this);
138 this.replicationPeers.getAllPeerIds();
139
140
141 int nbWorkers = conf.getInt("replication.executor.workers", 1);
142
143
144 this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
145 100, TimeUnit.MILLISECONDS,
146 new LinkedBlockingQueue<Runnable>());
147 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
148 tfb.setNameFormat("ReplicationExecutor-%d");
149 tfb.setDaemon(true);
150 this.executor.setThreadFactory(tfb.build());
151 this.rand = new Random();
152 }
153
154
155
156
157
158
159
160
161
162
163
164
165 public void logPositionAndCleanOldLogs(Path log, String id, long position,
166 boolean queueRecovered, boolean holdLogInZK) {
167 String fileName = log.getName();
168 this.replicationQueues.setLogPosition(id, fileName, position);
169 if (holdLogInZK) {
170 return;
171 }
172 cleanOldLogs(fileName, id, queueRecovered);
173 }
174
175
176
177
178
179
180
181
182 public void cleanOldLogs(String key, String id, boolean queueRecovered) {
183 if (queueRecovered) {
184 SortedSet<String> hlogs = hlogsByIdRecoveredQueues.get(id);
185 if (hlogs != null && !hlogs.first().equals(key)) {
186 cleanOldLogs(hlogs, key, id);
187 }
188 } else {
189 synchronized (this.hlogsById) {
190 SortedSet<String> hlogs = hlogsById.get(id);
191 if (!hlogs.first().equals(key)) {
192 cleanOldLogs(hlogs, key, id);
193 }
194 }
195 }
196 }
197
198 private void cleanOldLogs(SortedSet<String> hlogs, String key, String id) {
199 SortedSet<String> hlogSet = hlogs.headSet(key);
200 LOG.debug("Removing " + hlogSet.size() + " logs in the list: " + hlogSet);
201 for (String hlog : hlogSet) {
202 this.replicationQueues.removeLog(id, hlog);
203 }
204 hlogSet.clear();
205 }
206
207
208
209
210
211 protected void init() throws IOException, ReplicationException {
212 for (String id : this.replicationPeers.getConnectedPeers()) {
213 addSource(id);
214 }
215 List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
216 if (currentReplicators == null || currentReplicators.size() == 0) {
217 return;
218 }
219 List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
220 LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
221 + otherRegionServers);
222
223
224 for (String rs : currentReplicators) {
225 if (!otherRegionServers.contains(rs)) {
226 transferQueues(rs);
227 }
228 }
229 }
230
231
232
233
234
235
236
237 protected ReplicationSourceInterface addSource(String id) throws IOException,
238 ReplicationException {
239 ReplicationSourceInterface src =
240 getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
241 this.replicationPeers, stopper, id, this.clusterId);
242 synchronized (this.hlogsById) {
243 this.sources.add(src);
244 this.hlogsById.put(id, new TreeSet<String>());
245
246 if (this.latestPath != null) {
247 String name = this.latestPath.getName();
248 this.hlogsById.get(id).add(name);
249 try {
250 this.replicationQueues.addLog(src.getPeerClusterZnode(), name);
251 } catch (ReplicationException e) {
252 String message =
253 "Cannot add log to queue when creating a new source, queueId="
254 + src.getPeerClusterZnode() + ", filename=" + name;
255 stopper.stop(message);
256 throw e;
257 }
258 src.enqueueLog(this.latestPath);
259 }
260 }
261 src.startup();
262 return src;
263 }
264
265
266
267
268
269 public void deleteSource(String peerId, boolean closeConnection) {
270 this.replicationQueues.removeQueue(peerId);
271 if (closeConnection) {
272 this.replicationPeers.disconnectFromPeer(peerId);
273 }
274 }
275
276
277
278
279 public void join() {
280 this.executor.shutdown();
281 if (this.sources.size() == 0) {
282 this.replicationQueues.removeAllQueues();
283 }
284 for (ReplicationSourceInterface source : this.sources) {
285 source.terminate("Region server is closing");
286 }
287 }
288
289
290
291
292
293 protected Map<String, SortedSet<String>> getHLogs() {
294 return Collections.unmodifiableMap(hlogsById);
295 }
296
297
298
299
300
301 protected Map<String, SortedSet<String>> getHlogsByIdRecoveredQueues() {
302 return Collections.unmodifiableMap(hlogsByIdRecoveredQueues);
303 }
304
305
306
307
308
309 public List<ReplicationSourceInterface> getSources() {
310 return this.sources;
311 }
312
313
314
315
316
317 public List<ReplicationSourceInterface> getOldSources() {
318 return this.oldsources;
319 }
320
321 void preLogRoll(Path newLog) throws IOException {
322 synchronized (this.hlogsById) {
323 String name = newLog.getName();
324 for (ReplicationSourceInterface source : this.sources) {
325 try {
326 this.replicationQueues.addLog(source.getPeerClusterZnode(), name);
327 } catch (ReplicationException e) {
328 throw new IOException("Cannot add log to replication queue with id="
329 + source.getPeerClusterZnode() + ", filename=" + name, e);
330 }
331 }
332 for (SortedSet<String> hlogs : this.hlogsById.values()) {
333 if (this.sources.isEmpty()) {
334
335
336 hlogs.clear();
337 }
338 hlogs.add(name);
339 }
340 }
341
342 this.latestPath = newLog;
343 }
344
345 void postLogRoll(Path newLog) throws IOException {
346
347 for (ReplicationSourceInterface source : this.sources) {
348 source.enqueueLog(newLog);
349 }
350 }
351
352
353
354
355
356
357
358
359
360
361
362 protected ReplicationSourceInterface getReplicationSource(final Configuration conf,
363 final FileSystem fs, final ReplicationSourceManager manager,
364 final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
365 final Stoppable stopper, final String peerId, final UUID clusterId) throws IOException {
366 ReplicationSourceInterface src;
367 try {
368 @SuppressWarnings("rawtypes")
369 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
370 ReplicationSource.class.getCanonicalName()));
371 src = (ReplicationSourceInterface) c.newInstance();
372 } catch (Exception e) {
373 LOG.warn("Passed replication source implementation throws errors, " +
374 "defaulting to ReplicationSource", e);
375 src = new ReplicationSource();
376
377 }
378 src.init(conf, fs, manager, replicationQueues, replicationPeers, stopper, peerId, clusterId);
379 return src;
380 }
381
382
383
384
385
386
387
388
389
390 private void transferQueues(String rsZnode) {
391 NodeFailoverWorker transfer =
392 new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
393 this.clusterId);
394 try {
395 this.executor.execute(transfer);
396 } catch (RejectedExecutionException ex) {
397 LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
398 }
399 }
400
401
402
403
404
405 public void closeRecoveredQueue(ReplicationSourceInterface src) {
406 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
407 this.oldsources.remove(src);
408 deleteSource(src.getPeerClusterZnode(), false);
409 this.hlogsByIdRecoveredQueues.remove(src.getPeerClusterZnode());
410 }
411
412
413
414
415
416
417 public void removePeer(String id) {
418 LOG.info("Closing the following queue " + id + ", currently have "
419 + sources.size() + " and another "
420 + oldsources.size() + " that were recovered");
421 String terminateMessage = "Replication stream was removed by a user";
422 ReplicationSourceInterface srcToRemove = null;
423 List<ReplicationSourceInterface> oldSourcesToDelete =
424 new ArrayList<ReplicationSourceInterface>();
425
426 for (ReplicationSourceInterface src : oldsources) {
427 if (id.equals(src.getPeerClusterId())) {
428 oldSourcesToDelete.add(src);
429 }
430 }
431 for (ReplicationSourceInterface src : oldSourcesToDelete) {
432 src.terminate(terminateMessage);
433 closeRecoveredQueue((src));
434 }
435 LOG.info("Number of deleted recovered sources for " + id + ": "
436 + oldSourcesToDelete.size());
437
438 for (ReplicationSourceInterface src : this.sources) {
439 if (id.equals(src.getPeerClusterId())) {
440 srcToRemove = src;
441 break;
442 }
443 }
444 if (srcToRemove == null) {
445 LOG.error("The queue we wanted to close is missing " + id);
446 return;
447 }
448 srcToRemove.terminate(terminateMessage);
449 this.sources.remove(srcToRemove);
450 deleteSource(id, true);
451 }
452
453 @Override
454 public void regionServerRemoved(String regionserver) {
455 transferQueues(regionserver);
456 }
457
458 @Override
459 public void peerRemoved(String peerId) {
460 removePeer(peerId);
461 }
462
463 @Override
464 public void peerListChanged(List<String> peerIds) {
465 for (String id : peerIds) {
466 try {
467 boolean added = this.replicationPeers.connectToPeer(id);
468 if (added) {
469 addSource(id);
470 }
471 } catch (Exception e) {
472 LOG.error("Error while adding a new peer", e);
473 }
474 }
475 }
476
477
478
479
480
481 class NodeFailoverWorker extends Thread {
482
483 private String rsZnode;
484 private final ReplicationQueues rq;
485 private final ReplicationPeers rp;
486 private final UUID clusterId;
487
488
489
490
491
492 public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
493 final ReplicationPeers replicationPeers, final UUID clusterId) {
494 super("Failover-for-"+rsZnode);
495 this.rsZnode = rsZnode;
496 this.rq = replicationQueues;
497 this.rp = replicationPeers;
498 this.clusterId = clusterId;
499 }
500
501 @Override
502 public void run() {
503 if (this.rq.isThisOurZnode(rsZnode)) {
504 return;
505 }
506
507
508 try {
509 Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
510 } catch (InterruptedException e) {
511 LOG.warn("Interrupted while waiting before transferring a queue.");
512 Thread.currentThread().interrupt();
513 }
514
515 if (stopper.isStopped()) {
516 LOG.info("Not transferring queue since we are shutting down");
517 return;
518 }
519 SortedMap<String, SortedSet<String>> newQueues = null;
520
521 newQueues = this.rq.claimQueues(rsZnode);
522
523
524 if (newQueues.isEmpty()) {
525
526
527 return;
528 }
529
530 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
531 String peerId = entry.getKey();
532 try {
533 ReplicationSourceInterface src =
534 getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
535 stopper, peerId, this.clusterId);
536 if (!this.rp.getConnectedPeers().contains((src.getPeerClusterId()))) {
537 src.terminate("Recovered queue doesn't belong to any current peer");
538 break;
539 }
540 oldsources.add(src);
541 SortedSet<String> hlogsSet = entry.getValue();
542 for (String hlog : hlogsSet) {
543 src.enqueueLog(new Path(oldLogDir, hlog));
544 }
545 src.startup();
546 hlogsByIdRecoveredQueues.put(peerId, hlogsSet);
547 } catch (IOException e) {
548
549 LOG.error("Failed creating a source", e);
550 }
551 }
552 }
553 }
554
555
556
557
558
559 public Path getOldLogDir() {
560 return this.oldLogDir;
561 }
562
563
564
565
566
567 public Path getLogDir() {
568 return this.logDir;
569 }
570
571
572
573
574
575 public FileSystem getFs() {
576 return this.fs;
577 }
578
579
580
581
582 public String getStats() {
583 StringBuffer stats = new StringBuffer();
584 for (ReplicationSourceInterface source : sources) {
585 stats.append("Normal source for cluster " + source.getPeerClusterId() + ": ");
586 stats.append(source.getStats() + "\n");
587 }
588 for (ReplicationSourceInterface oldSource : oldsources) {
589 stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerClusterId() + ": ");
590 stats.append(oldSource.getStats()+ "\n");
591 }
592 return stats.toString();
593 }
594 }