1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver.handler;
20
21 import java.io.IOException;
22 import java.util.concurrent.atomic.AtomicInteger;
23 import java.util.concurrent.atomic.AtomicLong;
24
25 import org.apache.commons.lang.mutable.MutableInt;
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.Server;
30 import org.apache.hadoop.hbase.ServerName;
31 import org.apache.hadoop.hbase.SplitLogCounters;
32 import org.apache.hadoop.hbase.SplitLogTask;
33 import org.apache.hadoop.hbase.executor.EventHandler;
34 import org.apache.hadoop.hbase.executor.EventType;
35 import org.apache.hadoop.hbase.master.SplitLogManager;
36 import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
37 import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
38 import org.apache.hadoop.hbase.util.CancelableProgressable;
39 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
40 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
41 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
42 import org.apache.zookeeper.KeeperException;
43
44
45
46
47 @InterfaceAudience.Private
48 public class HLogSplitterHandler extends EventHandler {
49 private static final Log LOG = LogFactory.getLog(HLogSplitterHandler.class);
50 private final ServerName serverName;
51 private final String curTask;
52 private final String wal;
53 private final ZooKeeperWatcher zkw;
54 private final CancelableProgressable reporter;
55 private final AtomicInteger inProgressTasks;
56 private final MutableInt curTaskZKVersion;
57 private final TaskExecutor splitTaskExecutor;
58
59 public HLogSplitterHandler(final Server server, String curTask,
60 final MutableInt curTaskZKVersion,
61 CancelableProgressable reporter,
62 AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor) {
63 super(server, EventType.RS_LOG_REPLAY);
64 this.curTask = curTask;
65 this.wal = ZKSplitLog.getFileName(curTask);
66 this.reporter = reporter;
67 this.inProgressTasks = inProgressTasks;
68 this.inProgressTasks.incrementAndGet();
69 this.serverName = server.getServerName();
70 this.zkw = server.getZooKeeper();
71 this.curTaskZKVersion = curTaskZKVersion;
72 this.splitTaskExecutor = splitTaskExecutor;
73 }
74
75 @Override
76 public void process() throws IOException {
77 long startTime = System.currentTimeMillis();
78 try {
79 Status status = this.splitTaskExecutor.exec(wal, reporter);
80 switch (status) {
81 case DONE:
82 endTask(zkw, new SplitLogTask.Done(this.serverName),
83 SplitLogCounters.tot_wkr_task_done, curTask, curTaskZKVersion.intValue());
84 break;
85 case PREEMPTED:
86 SplitLogCounters.tot_wkr_preempt_task.incrementAndGet();
87 LOG.warn("task execution prempted " + wal);
88 break;
89 case ERR:
90 if (server != null && !server.isStopped()) {
91 endTask(zkw, new SplitLogTask.Err(this.serverName),
92 SplitLogCounters.tot_wkr_task_err, curTask, curTaskZKVersion.intValue());
93 break;
94 }
95
96
97
98 case RESIGNED:
99 if (server != null && server.isStopped()) {
100 LOG.info("task execution interrupted because worker is exiting " + curTask);
101 }
102 endTask(zkw, new SplitLogTask.Resigned(this.serverName),
103 SplitLogCounters.tot_wkr_task_resigned, curTask, curTaskZKVersion.intValue());
104 break;
105 }
106 } finally {
107 LOG.info("worker " + serverName + " done with task " + curTask + " in "
108 + (System.currentTimeMillis() - startTime) + "ms");
109 this.inProgressTasks.decrementAndGet();
110 }
111 }
112
113
114
115
116
117
118
119 public static void endTask(ZooKeeperWatcher zkw, SplitLogTask slt, AtomicLong ctr, String task,
120 int taskZKVersion) {
121 try {
122 if (ZKUtil.setData(zkw, task, slt.toByteArray(), taskZKVersion)) {
123 LOG.info("successfully transitioned task " + task + " to final state " + slt);
124 ctr.incrementAndGet();
125 return;
126 }
127 LOG.warn("failed to transistion task " + task + " to end state " + slt
128 + " because of version mismatch ");
129 } catch (KeeperException.BadVersionException bve) {
130 LOG.warn("transisition task " + task + " to " + slt
131 + " failed because of version mismatch", bve);
132 } catch (KeeperException.NoNodeException e) {
133 LOG.fatal(
134 "logic error - end task " + task + " " + slt
135 + " failed because task doesn't exist", e);
136 } catch (KeeperException e) {
137 LOG.warn("failed to end task, " + task + " " + slt, e);
138 }
139 SplitLogCounters.tot_wkr_final_transition_failed.incrementAndGet();
140 }
141 }