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