1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.coordination;
20
21 import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.CHECK;
22 import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.FORCE;
23 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.DELETED;
24 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.FAILURE;
25 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;
26 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
27
28 import java.io.IOException;
29 import java.io.InterruptedIOException;
30 import java.util.List;
31 import java.util.Set;
32 import java.util.concurrent.ConcurrentMap;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.CoordinatedStateManager;
40 import org.apache.hadoop.hbase.HConstants;
41 import org.apache.hadoop.hbase.HRegionInfo;
42 import org.apache.hadoop.hbase.Server;
43 import org.apache.hadoop.hbase.ServerName;
44 import org.apache.hadoop.hbase.SplitLogCounters;
45 import org.apache.hadoop.hbase.SplitLogTask;
46 import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination.TaskFinisher.Status;
47 import org.apache.hadoop.hbase.exceptions.DeserializationException;
48 import org.apache.hadoop.hbase.io.hfile.HFile;
49 import org.apache.hadoop.hbase.master.MasterServices;
50 import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
51 import org.apache.hadoop.hbase.master.SplitLogManager.Task;
52 import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
53 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
54 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
55 import org.apache.hadoop.hbase.wal.WALSplitter;
56 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
57 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
58 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
59 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
60 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
61 import org.apache.hadoop.util.StringUtils;
62 import org.apache.zookeeper.AsyncCallback;
63 import org.apache.zookeeper.CreateMode;
64 import org.apache.zookeeper.KeeperException;
65 import org.apache.zookeeper.KeeperException.NoNodeException;
66 import org.apache.zookeeper.ZooDefs.Ids;
67 import org.apache.zookeeper.data.Stat;
68
69
70
71
72
73 @InterfaceAudience.Private
74 public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
75 SplitLogManagerCoordination {
76
77 public static class ZkSplitLogManagerDetails extends SplitLogManagerDetails {
78
79 ZkSplitLogManagerDetails(ConcurrentMap<String, Task> tasks, MasterServices master,
80 Set<String> failedDeletions, ServerName serverName) {
81 super(tasks, master, failedDeletions, serverName);
82 }
83 }
84
85 public static final int DEFAULT_TIMEOUT = 120000;
86 public static final int DEFAULT_ZK_RETRIES = 3;
87 public static final int DEFAULT_MAX_RESUBMIT = 3;
88
89 private static final Log LOG = LogFactory.getLog(SplitLogManagerCoordination.class);
90
91 private Server server;
92 private long zkretries;
93 private long resubmitThreshold;
94 private long timeout;
95 private TaskFinisher taskFinisher;
96
97 SplitLogManagerDetails details;
98
99
100
101 private volatile long lastRecoveringNodeCreationTime = 0;
102 private Configuration conf;
103 public boolean ignoreZKDeleteForTesting = false;
104
105 private RecoveryMode recoveryMode;
106
107 private boolean isDrainingDone = false;
108
109 public ZKSplitLogManagerCoordination(final CoordinatedStateManager manager,
110 ZooKeeperWatcher watcher) {
111 super(watcher);
112 taskFinisher = new TaskFinisher() {
113 @Override
114 public Status finish(ServerName workerName, String logfile) {
115 try {
116 WALSplitter.finishSplitLogFile(logfile, manager.getServer().getConfiguration());
117 } catch (IOException e) {
118 LOG.warn("Could not finish splitting of log file " + logfile, e);
119 return Status.ERR;
120 }
121 return Status.DONE;
122 }
123 };
124 this.server = manager.getServer();
125 this.conf = server.getConfiguration();
126 }
127
128 @Override
129 public void init() throws IOException {
130 this.zkretries = conf.getLong("hbase.splitlog.zk.retries", DEFAULT_ZK_RETRIES);
131 this.resubmitThreshold = conf.getLong("hbase.splitlog.max.resubmit", DEFAULT_MAX_RESUBMIT);
132 this.timeout = conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT, DEFAULT_TIMEOUT);
133 setRecoveryMode(true);
134 if (this.watcher != null) {
135 this.watcher.registerListener(this);
136 lookForOrphans();
137 }
138 }
139
140 @Override
141 public String prepareTask(String taskname) {
142 return ZKSplitLog.getEncodedNodeName(watcher, taskname);
143 }
144
145 @Override
146 public int remainingTasksInCoordination() {
147 int count = 0;
148 try {
149 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
150 if (tasks != null) {
151 int listSize = tasks.size();
152 for (int i = 0; i < listSize; i++) {
153 if (!ZKSplitLog.isRescanNode(watcher, tasks.get(i))) {
154 count++;
155 }
156 }
157 }
158 } catch (KeeperException ke) {
159 LOG.warn("Failed to check remaining tasks", ke);
160 count = -1;
161 }
162 return count;
163 }
164
165
166
167
168
169
170
171
172 private void handleUnassignedTask(String path) {
173 if (ZKSplitLog.isRescanNode(watcher, path)) {
174 return;
175 }
176 Task task = findOrCreateOrphanTask(path);
177 if (task.isOrphan() && (task.incarnation == 0)) {
178 LOG.info("resubmitting unassigned orphan task " + path);
179
180
181 resubmitTask(path, task, FORCE);
182 }
183 }
184
185 @Override
186 public void deleteTask(String path) {
187 deleteNode(path, zkretries);
188 }
189
190 @Override
191 public boolean resubmitTask(String path, Task task, ResubmitDirective directive) {
192
193 if (task.status != IN_PROGRESS) {
194 return false;
195 }
196 int version;
197 if (directive != FORCE) {
198
199
200
201
202
203 final long time = EnvironmentEdgeManager.currentTime() - task.last_update;
204 final boolean alive =
205 details.getMaster().getServerManager() != null ? details.getMaster().getServerManager()
206 .isServerOnline(task.cur_worker_name) : true;
207 if (alive && time < timeout) {
208 LOG.trace("Skipping the resubmit of " + task.toString() + " because the server "
209 + task.cur_worker_name + " is not marked as dead, we waited for " + time
210 + " while the timeout is " + timeout);
211 return false;
212 }
213
214 if (task.unforcedResubmits.get() >= resubmitThreshold) {
215 if (!task.resubmitThresholdReached) {
216 task.resubmitThresholdReached = true;
217 SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
218 LOG.info("Skipping resubmissions of task " + path + " because threshold "
219 + resubmitThreshold + " reached");
220 }
221 return false;
222 }
223
224 version = task.last_version;
225 } else {
226 SplitLogCounters.tot_mgr_resubmit_force.incrementAndGet();
227 version = -1;
228 }
229 LOG.info("resubmitting task " + path);
230 task.incarnation++;
231 boolean result = resubmit(this.details.getServerName(), path, version);
232 if (!result) {
233 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTime());
234 return false;
235 }
236
237 if (directive != FORCE) {
238 task.unforcedResubmits.incrementAndGet();
239 }
240 task.setUnassigned();
241 rescan(Long.MAX_VALUE);
242 SplitLogCounters.tot_mgr_resubmit.incrementAndGet();
243 return true;
244 }
245
246
247 @Override
248 public void checkTasks() {
249 rescan(Long.MAX_VALUE);
250 };
251
252
253
254
255 private void rescan(long retries) {
256
257
258
259
260
261
262
263 SplitLogTask slt = new SplitLogTask.Done(this.details.getServerName(), getRecoveryMode());
264 this.watcher
265 .getRecoverableZooKeeper()
266 .getZooKeeper()
267 .create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
268 CreateMode.EPHEMERAL_SEQUENTIAL, new CreateRescanAsyncCallback(), Long.valueOf(retries));
269 }
270
271 @Override
272 public void submitTask(String path) {
273 createNode(path, zkretries);
274 }
275
276 @Override
277 public void checkTaskStillAvailable(String path) {
278
279 this.watcher
280 .getRecoverableZooKeeper()
281 .getZooKeeper()
282 .getData(path, this.watcher, new GetDataAsyncCallback(),
283 Long.valueOf(-1)
284 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
285 }
286
287
288
289
290
291
292
293
294 @Override
295 public void removeRecoveringRegions(final Set<String> recoveredServerNameSet,
296 Boolean isMetaRecovery)
297 throws IOException {
298 final String metaEncodeRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedName();
299 int count = 0;
300 try {
301 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
302 if (tasks != null) {
303 int listSize = tasks.size();
304 for (int i = 0; i < listSize; i++) {
305 if (!ZKSplitLog.isRescanNode(watcher, tasks.get(i))) {
306 count++;
307 }
308 }
309 }
310 if (count == 0 && this.details.getMaster().isInitialized()
311 && !this.details.getMaster().getServerManager().areDeadServersInProgress()) {
312
313 ZKSplitLog.deleteRecoveringRegionZNodes(watcher, null);
314
315
316 lastRecoveringNodeCreationTime = Long.MAX_VALUE;
317 } else if (!recoveredServerNameSet.isEmpty()) {
318
319 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
320 if (regions != null) {
321 int listSize = regions.size();
322 if (LOG.isDebugEnabled()) {
323 LOG.debug("Processing recovering " + regions + " and servers " +
324 recoveredServerNameSet + ", isMetaRecovery=" + isMetaRecovery);
325 }
326 for (int i = 0; i < listSize; i++) {
327 String region = regions.get(i);
328 if (isMetaRecovery != null) {
329 if ((isMetaRecovery && !region.equalsIgnoreCase(metaEncodeRegionName))
330 || (!isMetaRecovery && region.equalsIgnoreCase(metaEncodeRegionName))) {
331
332
333 continue;
334 }
335 }
336 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
337 List<String> failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
338 if (failedServers == null || failedServers.isEmpty()) {
339 ZKUtil.deleteNode(watcher, nodePath);
340 continue;
341 }
342 if (recoveredServerNameSet.containsAll(failedServers)) {
343 ZKUtil.deleteNodeRecursively(watcher, nodePath);
344 } else {
345 int tmpFailedServerSize = failedServers.size();
346 for (int j = 0; j < tmpFailedServerSize; j++) {
347 String failedServer = failedServers.get(j);
348 if (recoveredServerNameSet.contains(failedServer)) {
349 String tmpPath = ZKUtil.joinZNode(nodePath, failedServer);
350 ZKUtil.deleteNode(watcher, tmpPath);
351 }
352 }
353 }
354 }
355 }
356 }
357 } catch (KeeperException ke) {
358 LOG.warn("removeRecoveringRegionsFromZK got zookeeper exception. Will retry", ke);
359 throw new IOException(ke);
360 }
361 }
362
363 private void deleteNode(String path, Long retries) {
364 SplitLogCounters.tot_mgr_node_delete_queued.incrementAndGet();
365
366
367
368 this.watcher.getRecoverableZooKeeper().getZooKeeper()
369 .delete(path, -1, new DeleteAsyncCallback(), retries);
370 }
371
372 private void deleteNodeSuccess(String path) {
373 if (ignoreZKDeleteForTesting) {
374 return;
375 }
376 Task task;
377 task = details.getTasks().remove(path);
378 if (task == null) {
379 if (ZKSplitLog.isRescanNode(watcher, path)) {
380 SplitLogCounters.tot_mgr_rescan_deleted.incrementAndGet();
381 }
382 SplitLogCounters.tot_mgr_missing_state_in_delete.incrementAndGet();
383 LOG.debug("deleted task without in memory state " + path);
384 return;
385 }
386 synchronized (task) {
387 task.status = DELETED;
388 task.notify();
389 }
390 SplitLogCounters.tot_mgr_task_deleted.incrementAndGet();
391 }
392
393 private void deleteNodeFailure(String path) {
394 LOG.info("Failed to delete node " + path + " and will retry soon.");
395 return;
396 }
397
398 private void createRescanSuccess(String path) {
399 SplitLogCounters.tot_mgr_rescan.incrementAndGet();
400 getDataSetWatch(path, zkretries);
401 }
402
403 private void createRescanFailure() {
404 LOG.fatal("logic failure, rescan failure must not happen");
405 }
406
407
408
409
410
411
412
413 private boolean needAbandonRetries(int statusCode, String action) {
414 if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
415 LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
416 + "action=" + action);
417 return true;
418 }
419 return false;
420 }
421
422 private void createNode(String path, Long retry_count) {
423 SplitLogTask slt = new SplitLogTask.Unassigned(details.getServerName(), getRecoveryMode());
424 ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(),
425 retry_count);
426 SplitLogCounters.tot_mgr_node_create_queued.incrementAndGet();
427 return;
428 }
429
430 private void createNodeSuccess(String path) {
431 LOG.debug("put up splitlog task at znode " + path);
432 getDataSetWatch(path, zkretries);
433 }
434
435 private void createNodeFailure(String path) {
436
437 LOG.warn("failed to create task node" + path);
438 setDone(path, FAILURE);
439 }
440
441 private void getDataSetWatch(String path, Long retry_count) {
442 this.watcher.getRecoverableZooKeeper().getZooKeeper()
443 .getData(path, this.watcher, new GetDataAsyncCallback(), retry_count);
444 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
445 }
446
447
448 private void getDataSetWatchSuccess(String path, byte[] data, int version)
449 throws DeserializationException {
450 if (data == null) {
451 if (version == Integer.MIN_VALUE) {
452
453 setDone(path, SUCCESS);
454 return;
455 }
456 SplitLogCounters.tot_mgr_null_data.incrementAndGet();
457 LOG.fatal("logic error - got null data " + path);
458 setDone(path, FAILURE);
459 return;
460 }
461 data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
462 SplitLogTask slt = SplitLogTask.parseFrom(data);
463 if (slt.isUnassigned()) {
464 LOG.debug("task not yet acquired " + path + " ver = " + version);
465 handleUnassignedTask(path);
466 } else if (slt.isOwned()) {
467 heartbeat(path, version, slt.getServerName());
468 } else if (slt.isResigned()) {
469 LOG.info("task " + path + " entered state: " + slt.toString());
470 resubmitOrFail(path, FORCE);
471 } else if (slt.isDone()) {
472 LOG.info("task " + path + " entered state: " + slt.toString());
473 if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
474 if (taskFinisher.finish(slt.getServerName(), ZKSplitLog.getFileName(path)) == Status.DONE) {
475 setDone(path, SUCCESS);
476 } else {
477 resubmitOrFail(path, CHECK);
478 }
479 } else {
480 setDone(path, SUCCESS);
481 }
482 } else if (slt.isErr()) {
483 LOG.info("task " + path + " entered state: " + slt.toString());
484 resubmitOrFail(path, CHECK);
485 } else {
486 LOG.fatal("logic error - unexpected zk state for path = " + path + " data = "
487 + slt.toString());
488 setDone(path, FAILURE);
489 }
490 }
491
492 private void resubmitOrFail(String path, ResubmitDirective directive) {
493 if (resubmitTask(path, findOrCreateOrphanTask(path), directive) == false) {
494 setDone(path, FAILURE);
495 }
496 }
497
498 private void getDataSetWatchFailure(String path) {
499 LOG.warn("failed to set data watch " + path);
500 setDone(path, FAILURE);
501 }
502
503 private void setDone(String path, TerminationStatus status) {
504 Task task = details.getTasks().get(path);
505 if (task == null) {
506 if (!ZKSplitLog.isRescanNode(watcher, path)) {
507 SplitLogCounters.tot_mgr_unacquired_orphan_done.incrementAndGet();
508 LOG.debug("unacquired orphan task is done " + path);
509 }
510 } else {
511 synchronized (task) {
512 if (task.status == IN_PROGRESS) {
513 if (status == SUCCESS) {
514 SplitLogCounters.tot_mgr_log_split_success.incrementAndGet();
515 LOG.info("Done splitting " + path);
516 } else {
517 SplitLogCounters.tot_mgr_log_split_err.incrementAndGet();
518 LOG.warn("Error splitting " + path);
519 }
520 task.status = status;
521 if (task.batch != null) {
522 synchronized (task.batch) {
523 if (status == SUCCESS) {
524 task.batch.done++;
525 } else {
526 task.batch.error++;
527 }
528 task.batch.notify();
529 }
530 }
531 }
532 }
533 }
534
535
536
537
538
539 deleteNode(path, zkretries);
540 return;
541 }
542
543 Task findOrCreateOrphanTask(String path) {
544 Task orphanTask = new Task();
545 Task task;
546 task = details.getTasks().putIfAbsent(path, orphanTask);
547 if (task == null) {
548 LOG.info("creating orphan task " + path);
549 SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
550 task = orphanTask;
551 }
552 return task;
553 }
554
555 private void heartbeat(String path, int new_version, ServerName workerName) {
556 Task task = findOrCreateOrphanTask(path);
557 if (new_version != task.last_version) {
558 if (task.isUnassigned()) {
559 LOG.info("task " + path + " acquired by " + workerName);
560 }
561 task.heartbeat(EnvironmentEdgeManager.currentTime(), new_version, workerName);
562 SplitLogCounters.tot_mgr_heartbeat.incrementAndGet();
563 } else {
564
565
566
567
568 }
569 return;
570 }
571
572 private void lookForOrphans() {
573 List<String> orphans;
574 try {
575 orphans = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.splitLogZNode);
576 if (orphans == null) {
577 LOG.warn("could not get children of " + this.watcher.splitLogZNode);
578 return;
579 }
580 } catch (KeeperException e) {
581 LOG.warn("could not get children of " + this.watcher.splitLogZNode + " "
582 + StringUtils.stringifyException(e));
583 return;
584 }
585 int rescan_nodes = 0;
586 int listSize = orphans.size();
587 for (int i = 0; i < listSize; i++) {
588 String path = orphans.get(i);
589 String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
590 if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
591 rescan_nodes++;
592 LOG.debug("found orphan rescan node " + path);
593 } else {
594 LOG.info("found orphan task " + path);
595 }
596 getDataSetWatch(nodepath, zkretries);
597 }
598 LOG.info("Found " + (orphans.size() - rescan_nodes) + " orphan tasks and " + rescan_nodes
599 + " rescan nodes");
600 }
601
602
603
604
605
606
607
608 @Override
609 public void markRegionsRecovering(final ServerName serverName, Set<HRegionInfo> userRegions)
610 throws IOException, InterruptedIOException {
611 this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTime();
612 for (HRegionInfo region : userRegions) {
613 String regionEncodeName = region.getEncodedName();
614 long retries = this.zkretries;
615
616 do {
617 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName);
618 long lastRecordedFlushedSequenceId = -1;
619 try {
620 long lastSequenceId =
621 this.details.getMaster().getServerManager()
622 .getLastFlushedSequenceId(regionEncodeName.getBytes());
623
624
625
626
627
628 byte[] data = ZKUtil.getData(this.watcher, nodePath);
629 if (data == null) {
630 ZKUtil
631 .createSetData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
632 } else {
633 lastRecordedFlushedSequenceId =
634 ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
635 if (lastRecordedFlushedSequenceId < lastSequenceId) {
636
637 ZKUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
638 }
639 }
640
641 nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName());
642 if (lastSequenceId <= lastRecordedFlushedSequenceId) {
643
644 lastSequenceId = lastRecordedFlushedSequenceId;
645 }
646 ZKUtil.createSetData(this.watcher, nodePath,
647 ZKUtil.regionSequenceIdsToByteArray(lastSequenceId, null));
648 if (LOG.isDebugEnabled()) {
649 LOG.debug("Marked " + regionEncodeName + " as recovering from " + serverName +
650 ": " + nodePath);
651 }
652
653 break;
654 } catch (KeeperException e) {
655
656 if (retries <= 1) {
657 throw new IOException(e);
658 }
659
660 try {
661 Thread.sleep(20);
662 } catch (InterruptedException e1) {
663 throw new InterruptedIOException();
664 }
665 } catch (InterruptedException e) {
666 throw new InterruptedIOException();
667 }
668 } while ((--retries) > 0);
669 }
670 }
671
672 @Override
673 public void nodeDataChanged(String path) {
674 Task task;
675 task = details.getTasks().get(path);
676 if (task != null || ZKSplitLog.isRescanNode(watcher, path)) {
677 if (task != null) {
678 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTime());
679 }
680 getDataSetWatch(path, zkretries);
681 }
682 }
683
684
685
686
687
688
689 @Override
690 public void removeStaleRecoveringRegions(final Set<String> knownFailedServers)
691 throws IOException, InterruptedIOException {
692
693 try {
694 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
695 if (tasks != null) {
696 int listSize = tasks.size();
697 for (int i = 0; i < listSize; i++) {
698 String t = tasks.get(i);
699 byte[] data;
700 try {
701 data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
702 } catch (InterruptedException e) {
703 throw new InterruptedIOException();
704 }
705 if (data != null) {
706 SplitLogTask slt = null;
707 try {
708 slt = SplitLogTask.parseFrom(data);
709 } catch (DeserializationException e) {
710 LOG.warn("Failed parse data for znode " + t, e);
711 }
712 if (slt != null && slt.isDone()) {
713 continue;
714 }
715 }
716
717 t = ZKSplitLog.getFileName(t);
718 ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(new Path(t));
719 if (serverName != null) {
720 knownFailedServers.add(serverName.getServerName());
721 } else {
722 LOG.warn("Found invalid WAL log file name:" + t);
723 }
724 }
725 }
726
727
728 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
729 if (regions != null) {
730 int listSize = regions.size();
731 for (int i = 0; i < listSize; i++) {
732 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regions.get(i));
733 List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
734 if (regionFailedServers == null || regionFailedServers.isEmpty()) {
735 ZKUtil.deleteNode(watcher, nodePath);
736 continue;
737 }
738 boolean needMoreRecovery = false;
739 int tmpFailedServerSize = regionFailedServers.size();
740 for (int j = 0; j < tmpFailedServerSize; j++) {
741 if (knownFailedServers.contains(regionFailedServers.get(j))) {
742 needMoreRecovery = true;
743 break;
744 }
745 }
746 if (!needMoreRecovery) {
747 ZKUtil.deleteNodeRecursively(watcher, nodePath);
748 }
749 }
750 }
751 } catch (KeeperException e) {
752 throw new IOException(e);
753 }
754 }
755
756 @Override
757 public synchronized boolean isReplaying() {
758 return this.recoveryMode == RecoveryMode.LOG_REPLAY;
759 }
760
761 @Override
762 public synchronized boolean isSplitting() {
763 return this.recoveryMode == RecoveryMode.LOG_SPLITTING;
764 }
765
766
767
768
769
770
771
772 @Override
773 public void setRecoveryMode(boolean isForInitialization) throws IOException {
774 synchronized(this) {
775 if (this.isDrainingDone) {
776
777
778 return;
779 }
780 }
781 if (this.watcher == null) {
782
783 synchronized(this) {
784 this.isDrainingDone = true;
785 this.recoveryMode = RecoveryMode.LOG_SPLITTING;
786 }
787 return;
788 }
789 boolean hasSplitLogTask = false;
790 boolean hasRecoveringRegions = false;
791 RecoveryMode previousRecoveryMode = RecoveryMode.UNKNOWN;
792 RecoveryMode recoveryModeInConfig =
793 (isDistributedLogReplay(conf)) ? RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING;
794
795
796 try {
797 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
798 if (regions != null && !regions.isEmpty()) {
799 hasRecoveringRegions = true;
800 previousRecoveryMode = RecoveryMode.LOG_REPLAY;
801 }
802 if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
803
804 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
805 if (tasks != null && !tasks.isEmpty()) {
806 hasSplitLogTask = true;
807 if (isForInitialization) {
808
809 int listSize = tasks.size();
810 for (int i = 0; i < listSize; i++) {
811 String task = tasks.get(i);
812 try {
813 byte[] data =
814 ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, task));
815 if (data == null) continue;
816 SplitLogTask slt = SplitLogTask.parseFrom(data);
817 previousRecoveryMode = slt.getMode();
818 if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
819
820
821
822 previousRecoveryMode = RecoveryMode.LOG_SPLITTING;
823 }
824 break;
825 } catch (DeserializationException e) {
826 LOG.warn("Failed parse data for znode " + task, e);
827 } catch (InterruptedException e) {
828 throw new InterruptedIOException();
829 }
830 }
831 }
832 }
833 }
834 } catch (KeeperException e) {
835 throw new IOException(e);
836 }
837
838 synchronized (this) {
839 if (this.isDrainingDone) {
840 return;
841 }
842 if (!hasSplitLogTask && !hasRecoveringRegions) {
843 this.isDrainingDone = true;
844 this.recoveryMode = recoveryModeInConfig;
845 return;
846 } else if (!isForInitialization) {
847
848 return;
849 }
850
851 if (previousRecoveryMode != RecoveryMode.UNKNOWN) {
852 this.isDrainingDone = (previousRecoveryMode == recoveryModeInConfig);
853 this.recoveryMode = previousRecoveryMode;
854 } else {
855 this.recoveryMode = recoveryModeInConfig;
856 }
857 }
858 }
859
860
861
862
863
864
865 private boolean isDistributedLogReplay(Configuration conf) {
866 boolean dlr =
867 conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
868 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
869 if (LOG.isDebugEnabled()) {
870 LOG.debug("Distributed log replay=" + dlr);
871 }
872 return dlr;
873 }
874
875 private boolean resubmit(ServerName serverName, String path, int version) {
876 try {
877
878 SplitLogTask slt =
879 new SplitLogTask.Unassigned(this.details.getServerName(), getRecoveryMode());
880 if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
881 LOG.debug("failed to resubmit task " + path + " version changed");
882 return false;
883 }
884 } catch (NoNodeException e) {
885 LOG.warn("failed to resubmit because znode doesn't exist " + path
886 + " task done (or forced done by removing the znode)");
887 try {
888 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
889 } catch (DeserializationException e1) {
890 LOG.debug("Failed to re-resubmit task " + path + " because of deserialization issue", e1);
891 return false;
892 }
893 return false;
894 } catch (KeeperException.BadVersionException e) {
895 LOG.debug("failed to resubmit task " + path + " version changed");
896 return false;
897 } catch (KeeperException e) {
898 SplitLogCounters.tot_mgr_resubmit_failed.incrementAndGet();
899 LOG.warn("failed to resubmit " + path, e);
900 return false;
901 }
902 return true;
903 }
904
905
906
907
908
909
910
911
912
913 public interface TaskFinisher {
914
915
916
917 enum Status {
918
919
920
921 DONE(),
922
923
924
925 ERR();
926 }
927
928
929
930
931
932
933
934
935
936
937 Status finish(ServerName workerName, String taskname);
938 }
939
940
941
942
943 public class CreateAsyncCallback implements AsyncCallback.StringCallback {
944 private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
945
946 @Override
947 public void processResult(int rc, String path, Object ctx, String name) {
948 SplitLogCounters.tot_mgr_node_create_result.incrementAndGet();
949 if (rc != 0) {
950 if (needAbandonRetries(rc, "Create znode " + path)) {
951 createNodeFailure(path);
952 return;
953 }
954 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
955
956
957
958
959
960
961 LOG.debug("found pre-existing znode " + path);
962 SplitLogCounters.tot_mgr_node_already_exists.incrementAndGet();
963 } else {
964 Long retry_count = (Long) ctx;
965 LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " + path
966 + " remaining retries=" + retry_count);
967 if (retry_count == 0) {
968 SplitLogCounters.tot_mgr_node_create_err.incrementAndGet();
969 createNodeFailure(path);
970 } else {
971 SplitLogCounters.tot_mgr_node_create_retry.incrementAndGet();
972 createNode(path, retry_count - 1);
973 }
974 return;
975 }
976 }
977 createNodeSuccess(path);
978 }
979 }
980
981
982
983
984 public class GetDataAsyncCallback implements AsyncCallback.DataCallback {
985 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
986
987 @Override
988 public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
989 SplitLogCounters.tot_mgr_get_data_result.incrementAndGet();
990 if (rc != 0) {
991 if (needAbandonRetries(rc, "GetData from znode " + path)) {
992 return;
993 }
994 if (rc == KeeperException.Code.NONODE.intValue()) {
995 SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
996 LOG.warn("task znode " + path + " vanished or not created yet.");
997
998
999
1000 return;
1001 }
1002 Long retry_count = (Long) ctx;
1003
1004 if (retry_count < 0) {
1005 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path
1006 + ". Ignoring error. No error handling. No retrying.");
1007 return;
1008 }
1009 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path
1010 + " remaining retries=" + retry_count);
1011 if (retry_count == 0) {
1012 SplitLogCounters.tot_mgr_get_data_err.incrementAndGet();
1013 getDataSetWatchFailure(path);
1014 } else {
1015 SplitLogCounters.tot_mgr_get_data_retry.incrementAndGet();
1016 getDataSetWatch(path, retry_count - 1);
1017 }
1018 return;
1019 }
1020 try {
1021 getDataSetWatchSuccess(path, data, stat.getVersion());
1022 } catch (DeserializationException e) {
1023 LOG.warn("Deserialization problem", e);
1024 }
1025 return;
1026 }
1027 }
1028
1029
1030
1031
1032 public class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
1033 private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
1034
1035 @Override
1036 public void processResult(int rc, String path, Object ctx) {
1037 SplitLogCounters.tot_mgr_node_delete_result.incrementAndGet();
1038 if (rc != 0) {
1039 if (needAbandonRetries(rc, "Delete znode " + path)) {
1040 details.getFailedDeletions().add(path);
1041 return;
1042 }
1043 if (rc != KeeperException.Code.NONODE.intValue()) {
1044 SplitLogCounters.tot_mgr_node_delete_err.incrementAndGet();
1045 Long retry_count = (Long) ctx;
1046 LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " + path
1047 + " remaining retries=" + retry_count);
1048 if (retry_count == 0) {
1049 LOG.warn("delete failed " + path);
1050 details.getFailedDeletions().add(path);
1051 deleteNodeFailure(path);
1052 } else {
1053 deleteNode(path, retry_count - 1);
1054 }
1055 return;
1056 } else {
1057 LOG.info(path + " does not exist. Either was created but deleted behind our"
1058 + " back by another pending delete OR was deleted"
1059 + " in earlier retry rounds. zkretries = " + ctx);
1060 }
1061 } else {
1062 LOG.debug("deleted " + path);
1063 }
1064 deleteNodeSuccess(path);
1065 }
1066 }
1067
1068
1069
1070
1071
1072
1073
1074 public class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
1075 private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
1076
1077 @Override
1078 public void processResult(int rc, String path, Object ctx, String name) {
1079 if (rc != 0) {
1080 if (needAbandonRetries(rc, "CreateRescan znode " + path)) {
1081 return;
1082 }
1083 Long retry_count = (Long) ctx;
1084 LOG.warn("rc=" + KeeperException.Code.get(rc) + " for " + path + " remaining retries="
1085 + retry_count);
1086 if (retry_count == 0) {
1087 createRescanFailure();
1088 } else {
1089 rescan(retry_count - 1);
1090 }
1091 return;
1092 }
1093
1094 createRescanSuccess(name);
1095 }
1096 }
1097
1098 @Override
1099 public void setDetails(SplitLogManagerDetails details) {
1100 this.details = details;
1101 }
1102
1103 @Override
1104 public SplitLogManagerDetails getDetails() {
1105 return details;
1106 }
1107
1108 @Override
1109 public synchronized RecoveryMode getRecoveryMode() {
1110 return recoveryMode;
1111 }
1112
1113 @Override
1114 public long getLastRecoveryTime() {
1115 return lastRecoveringNodeCreationTime;
1116 }
1117
1118
1119
1120
1121 public void setIgnoreDeleteForTesting(boolean b) {
1122 ignoreZKDeleteForTesting = b;
1123 }
1124 }