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