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 void preLogRoll(Path newLog) throws IOException {
289
290 synchronized (this.hlogsById) {
291 String name = newLog.getName();
292 for (ReplicationSourceInterface source : this.sources) {
293 try {
294 this.replicationQueues.addLog(source.getPeerClusterZnode(), name);
295 } catch (ReplicationException e) {
296 throw new IOException("Cannot add log to replication queue with id="
297 + source.getPeerClusterZnode() + ", filename=" + name, e);
298 }
299 }
300 for (SortedSet<String> hlogs : this.hlogsById.values()) {
301 if (this.sources.isEmpty()) {
302
303
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
315 for (ReplicationSourceInterface source : this.sources) {
316 source.enqueueLog(newLog);
317 }
318 }
319
320
321
322
323
324
325
326
327
328
329
330 protected ReplicationSourceInterface getReplicationSource(final Configuration conf,
331 final FileSystem fs, final ReplicationSourceManager manager,
332 final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
333 final Stoppable stopper, final String peerId, final UUID clusterId) throws IOException {
334 ReplicationSourceInterface src;
335 try {
336 @SuppressWarnings("rawtypes")
337 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
338 ReplicationSource.class.getCanonicalName()));
339 src = (ReplicationSourceInterface) c.newInstance();
340 } catch (Exception e) {
341 LOG.warn("Passed replication source implementation throws errors, " +
342 "defaulting to ReplicationSource", e);
343 src = new ReplicationSource();
344
345 }
346 src.init(conf, fs, manager, replicationQueues, replicationPeers, stopper, peerId, clusterId);
347 return src;
348 }
349
350
351
352
353
354
355
356
357
358 private void transferQueues(String rsZnode) {
359 NodeFailoverWorker transfer =
360 new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
361 this.clusterId);
362 try {
363 this.executor.execute(transfer);
364 } catch (RejectedExecutionException ex) {
365 LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
366 }
367 }
368
369
370
371
372
373 public void closeRecoveredQueue(ReplicationSourceInterface src) {
374 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
375 this.oldsources.remove(src);
376 deleteSource(src.getPeerClusterZnode(), false);
377 }
378
379
380
381
382
383
384 public void removePeer(String id) {
385 LOG.info("Closing the following queue " + id + ", currently have "
386 + sources.size() + " and another "
387 + oldsources.size() + " that were recovered");
388 String terminateMessage = "Replication stream was removed by a user";
389 ReplicationSourceInterface srcToRemove = null;
390 List<ReplicationSourceInterface> oldSourcesToDelete =
391 new ArrayList<ReplicationSourceInterface>();
392
393 for (ReplicationSourceInterface src : oldsources) {
394 if (id.equals(src.getPeerClusterId())) {
395 oldSourcesToDelete.add(src);
396 }
397 }
398 for (ReplicationSourceInterface src : oldSourcesToDelete) {
399 src.terminate(terminateMessage);
400 closeRecoveredQueue((src));
401 }
402 LOG.info("Number of deleted recovered sources for " + id + ": "
403 + oldSourcesToDelete.size());
404
405 for (ReplicationSourceInterface src : this.sources) {
406 if (id.equals(src.getPeerClusterId())) {
407 srcToRemove = src;
408 break;
409 }
410 }
411 if (srcToRemove == null) {
412 LOG.error("The queue we wanted to close is missing " + id);
413 return;
414 }
415 srcToRemove.terminate(terminateMessage);
416 this.sources.remove(srcToRemove);
417 deleteSource(id, true);
418 }
419
420 @Override
421 public void regionServerRemoved(String regionserver) {
422 transferQueues(regionserver);
423 }
424
425 @Override
426 public void peerRemoved(String peerId) {
427 removePeer(peerId);
428 }
429
430 @Override
431 public void peerListChanged(List<String> peerIds) {
432 for (String id : peerIds) {
433 try {
434 boolean added = this.replicationPeers.connectToPeer(id);
435 if (added) {
436 addSource(id);
437 }
438 } catch (Exception e) {
439 LOG.error("Error while adding a new peer", e);
440 }
441 }
442 }
443
444
445
446
447
448 class NodeFailoverWorker extends Thread {
449
450 private String rsZnode;
451 private final ReplicationQueues rq;
452 private final ReplicationPeers rp;
453 private final UUID clusterId;
454
455
456
457
458
459 public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
460 final ReplicationPeers replicationPeers, final UUID clusterId) {
461 super("Failover-for-"+rsZnode);
462 this.rsZnode = rsZnode;
463 this.rq = replicationQueues;
464 this.rp = replicationPeers;
465 this.clusterId = clusterId;
466 }
467
468 @Override
469 public void run() {
470
471
472 try {
473 Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
474 } catch (InterruptedException e) {
475 LOG.warn("Interrupted while waiting before transferring a queue.");
476 Thread.currentThread().interrupt();
477 }
478
479 if (stopper.isStopped()) {
480 LOG.info("Not transferring queue since we are shutting down");
481 return;
482 }
483 SortedMap<String, SortedSet<String>> newQueues = null;
484
485 newQueues = this.rq.claimQueues(rsZnode);
486
487
488 if (newQueues.isEmpty()) {
489
490
491 return;
492 }
493
494 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
495 String peerId = entry.getKey();
496 try {
497 ReplicationSourceInterface src =
498 getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
499 stopper, peerId, this.clusterId);
500 if (!this.rp.getConnectedPeers().contains((src.getPeerClusterId()))) {
501 src.terminate("Recovered queue doesn't belong to any current peer");
502 break;
503 }
504 oldsources.add(src);
505 for (String hlog : entry.getValue()) {
506 src.enqueueLog(new Path(oldLogDir, hlog));
507 }
508 src.startup();
509 } catch (IOException e) {
510
511 LOG.error("Failed creating a source", e);
512 }
513 }
514 }
515 }
516
517
518
519
520
521 public Path getOldLogDir() {
522 return this.oldLogDir;
523 }
524
525
526
527
528
529 public Path getLogDir() {
530 return this.logDir;
531 }
532
533
534
535
536
537 public FileSystem getFs() {
538 return this.fs;
539 }
540
541
542
543
544 public String getStats() {
545 StringBuffer stats = new StringBuffer();
546 for (ReplicationSourceInterface source : sources) {
547 stats.append("Normal source for cluster " + source.getPeerClusterId() + ": ");
548 stats.append(source.getStats() + "\n");
549 }
550 for (ReplicationSourceInterface oldSource : oldsources) {
551 stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerClusterId() + ": ");
552 stats.append(oldSource.getStats()+ "\n");
553 }
554 return stats.toString();
555 }
556 }