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.regionserver;
21
22 import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
23
24 import java.io.IOException;
25 import java.io.InterruptedIOException;
26 import java.util.List;
27 import java.util.concurrent.atomic.AtomicLong;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.hbase.master.SplitLogManager;
35 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
36 import org.apache.hadoop.hbase.util.CancelableProgressable;
37 import org.apache.hadoop.hbase.util.FSUtils;
38 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
39 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
40 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
41 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
42 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
43 import org.apache.hadoop.util.StringUtils;
44 import org.apache.zookeeper.AsyncCallback;
45 import org.apache.zookeeper.KeeperException;
46 import org.apache.zookeeper.data.Stat;
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68 public class SplitLogWorker extends ZooKeeperListener implements Runnable {
69 private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
70
71 Thread worker;
72 private final String serverName;
73 private final TaskExecutor executor;
74 private long zkretries;
75
76 private Object taskReadyLock = new Object();
77 volatile int taskReadySeq = 0;
78 private volatile String currentTask = null;
79 private int currentVersion;
80 private volatile boolean exitWorker;
81 private Object grabTaskLock = new Object();
82 private boolean workerInGrabTask = false;
83
84
85 public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf,
86 String serverName, TaskExecutor executor) {
87 super(watcher);
88 this.serverName = serverName;
89 this.executor = executor;
90 this.zkretries = conf.getLong("hbase.splitlog.zk.retries", 3);
91 }
92
93 public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf,
94 final String serverName) {
95 this(watcher, conf, serverName, new TaskExecutor () {
96 @Override
97 public Status exec(String filename, CancelableProgressable p) {
98 Path rootdir;
99 FileSystem fs;
100 try {
101 rootdir = FSUtils.getRootDir(conf);
102 fs = rootdir.getFileSystem(conf);
103 } catch (IOException e) {
104 LOG.warn("could not find root dir or fs", e);
105 return Status.RESIGNED;
106 }
107
108
109
110 try {
111 String tmpname =
112 ZKSplitLog.getSplitLogDirTmpComponent(serverName, filename);
113 if (HLogSplitter.splitLogFileToTemp(rootdir, tmpname,
114 fs.getFileStatus(new Path(filename)), fs, conf, p) == false) {
115 return Status.PREEMPTED;
116 }
117 } catch (InterruptedIOException iioe) {
118 LOG.warn("log splitting of " + filename + " interrupted, resigning",
119 iioe);
120 return Status.RESIGNED;
121 } catch (IOException e) {
122 Throwable cause = e.getCause();
123 if (cause instanceof InterruptedException) {
124 LOG.warn("log splitting of " + filename + " interrupted, resigning",
125 e);
126 return Status.RESIGNED;
127 }
128 LOG.warn("log splitting of " + filename + " failed, returning error",
129 e);
130 return Status.ERR;
131 }
132 return Status.DONE;
133 }
134 });
135 }
136
137 @Override
138 public void run() {
139 try {
140 LOG.info("SplitLogWorker " + this.serverName + " starting");
141 this.watcher.registerListener(this);
142 int res;
143
144 res = -1;
145 while (res == -1) {
146 try {
147 res = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
148 } catch (KeeperException e) {
149
150 LOG.warn("Exception when checking for " + watcher.splitLogZNode +
151 " ... retrying", e);
152 }
153 if (res == -1) {
154 try {
155 LOG.info(watcher.splitLogZNode + " znode does not exist," +
156 " waiting for master to create one");
157 Thread.sleep(1000);
158 } catch (InterruptedException e) {
159 LOG.debug("Interrupted while waiting for " + watcher.splitLogZNode);
160 assert exitWorker == true;
161 }
162 }
163 }
164
165 taskLoop();
166 } catch (Throwable t) {
167
168
169 LOG.error("unexpected error ", t);
170 } finally {
171 LOG.info("SplitLogWorker " + this.serverName + " exiting");
172 }
173 }
174
175
176
177
178
179
180
181
182
183 private void taskLoop() {
184 while (true) {
185 int seq_start = taskReadySeq;
186 List<String> paths = getTaskList();
187 if (paths == null) {
188 LOG.warn("Could not get tasks, did someone remove " +
189 this.watcher.splitLogZNode + " ... worker thread exiting.");
190 return;
191 }
192 int offset = (int)(Math.random() * paths.size());
193 for (int i = 0; i < paths.size(); i ++) {
194 int idx = (i + offset) % paths.size();
195
196
197 grabTask(ZKUtil.joinZNode(watcher.splitLogZNode, paths.get(idx)));
198 if (exitWorker == true) {
199 return;
200 }
201 }
202 synchronized (taskReadyLock) {
203 while (seq_start == taskReadySeq) {
204 try {
205 taskReadyLock.wait();
206 } catch (InterruptedException e) {
207 LOG.info("SplitLogWorker interrupted while waiting for task," +
208 " exiting: " + e.toString());
209 assert exitWorker == true;
210 return;
211 }
212 }
213 }
214 }
215 }
216
217
218
219
220
221
222 private void grabTask(String path) {
223 Stat stat = new Stat();
224 long t = -1;
225 byte[] data;
226 synchronized (grabTaskLock) {
227 currentTask = path;
228 workerInGrabTask = true;
229 if (Thread.interrupted()) {
230 return;
231 }
232 }
233 try {
234 try {
235 if ((data = ZKUtil.getDataNoWatch(this.watcher, path, stat)) == null) {
236 tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
237 return;
238 }
239 } catch (KeeperException e) {
240 LOG.warn("Failed to get data for znode " + path, e);
241 tot_wkr_failed_to_grab_task_exception.incrementAndGet();
242 return;
243 }
244 if (TaskState.TASK_UNASSIGNED.equals(data) == false) {
245 tot_wkr_failed_to_grab_task_owned.incrementAndGet();
246 return;
247 }
248
249 currentVersion = stat.getVersion();
250 if (ownTask(true) == false) {
251 tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
252 return;
253 }
254
255 if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
256 endTask(TaskState.TASK_DONE, tot_wkr_task_acquired_rescan);
257 return;
258 }
259 LOG.info("worker " + serverName + " acquired task " + path);
260 tot_wkr_task_acquired.incrementAndGet();
261 getDataSetWatchAsync();
262
263 t = System.currentTimeMillis();
264 TaskExecutor.Status status;
265
266 status = executor.exec(ZKSplitLog.getFileName(currentTask),
267 new CancelableProgressable() {
268
269 @Override
270 public boolean progress() {
271 if (ownTask(false) == false) {
272 LOG.warn("Failed to heartbeat the task" + currentTask);
273 return false;
274 }
275 return true;
276 }
277 });
278 switch (status) {
279 case DONE:
280 endTask(TaskState.TASK_DONE, tot_wkr_task_done);
281 break;
282 case PREEMPTED:
283 tot_wkr_preempt_task.incrementAndGet();
284 LOG.warn("task execution prempted " + path);
285 break;
286 case ERR:
287 if (!exitWorker) {
288 endTask(TaskState.TASK_ERR, tot_wkr_task_err);
289 break;
290 }
291
292
293
294 case RESIGNED:
295 if (exitWorker) {
296 LOG.info("task execution interrupted because worker is exiting " +
297 path);
298 endTask(TaskState.TASK_RESIGNED, tot_wkr_task_resigned);
299 } else {
300 tot_wkr_preempt_task.incrementAndGet();
301 LOG.info("task execution interrupted via zk by manager " +
302 path);
303 }
304 break;
305 }
306 } finally {
307 if (t > 0) {
308 LOG.info("worker " + serverName + " done with task " + path +
309 " in " + (System.currentTimeMillis() - t) + "ms");
310 }
311 synchronized (grabTaskLock) {
312 workerInGrabTask = false;
313
314
315 Thread.interrupted();
316 }
317 }
318 return;
319 }
320
321
322
323
324
325
326
327
328
329
330 private boolean ownTask(boolean isFirstTime) {
331 try {
332 Stat stat = this.watcher.getRecoverableZooKeeper().setData(currentTask,
333 TaskState.TASK_OWNED.get(serverName), currentVersion);
334 if (stat == null) {
335 LOG.warn("zk.setData() returned null for path " + currentTask);
336 tot_wkr_task_heartbeat_failed.incrementAndGet();
337 return (false);
338 }
339 currentVersion = stat.getVersion();
340 tot_wkr_task_heartbeat.incrementAndGet();
341 return (true);
342 } catch (KeeperException e) {
343 if (!isFirstTime) {
344 if (e.code().equals(KeeperException.Code.NONODE)) {
345 LOG.warn("NONODE failed to assert ownership for " + currentTask, e);
346 } else if (e.code().equals(KeeperException.Code.BADVERSION)) {
347 LOG.warn("BADVERSION failed to assert ownership for " +
348 currentTask, e);
349 } else {
350 LOG.warn("failed to assert ownership for " + currentTask, e);
351 }
352 }
353 } catch (InterruptedException e1) {
354 LOG.warn("Interrupted while trying to assert ownership of " +
355 currentTask + " " + StringUtils.stringifyException(e1));
356 Thread.currentThread().interrupt();
357 }
358 tot_wkr_task_heartbeat_failed.incrementAndGet();
359 return (false);
360 }
361
362
363
364
365
366
367
368 private void endTask(ZKSplitLog.TaskState ts, AtomicLong ctr) {
369 String path = currentTask;
370 currentTask = null;
371 try {
372 if (ZKUtil.setData(this.watcher, path, ts.get(serverName),
373 currentVersion)) {
374 LOG.info("successfully transitioned task " + path +
375 " to final state " + ts);
376 ctr.incrementAndGet();
377 return;
378 }
379 LOG.warn("failed to transistion task " + path + " to end state " + ts +
380 " because of version mismatch ");
381 } catch (KeeperException.BadVersionException bve) {
382 LOG.warn("transisition task " + path + " to " + ts +
383 " failed because of version mismatch", bve);
384 } catch (KeeperException.NoNodeException e) {
385 LOG.fatal("logic error - end task " + path + " " + ts +
386 " failed because task doesn't exist", e);
387 } catch (KeeperException e) {
388 LOG.warn("failed to end task, " + path + " " + ts, e);
389 }
390 tot_wkr_final_transistion_failed.incrementAndGet();
391 return;
392 }
393
394 void getDataSetWatchAsync() {
395 this.watcher.getRecoverableZooKeeper().getZooKeeper().
396 getData(currentTask, this.watcher,
397 new GetDataAsyncCallback(), null);
398 tot_wkr_get_data_queued.incrementAndGet();
399 }
400
401 void getDataSetWatchSuccess(String path, byte[] data) {
402 synchronized (grabTaskLock) {
403 if (workerInGrabTask) {
404
405 String taskpath = currentTask;
406 if (taskpath != null && taskpath.equals(path)) {
407
408
409
410
411
412
413 if (! TaskState.TASK_OWNED.equals(data, serverName) &&
414 ! TaskState.TASK_DONE.equals(data, serverName) &&
415 ! TaskState.TASK_ERR.equals(data, serverName) &&
416 ! TaskState.TASK_RESIGNED.equals(data, serverName)) {
417 LOG.info("task " + taskpath + " preempted from " +
418 serverName + ", current task state and owner=" +
419 new String(data));
420 stopTask();
421 }
422 }
423 }
424 }
425 }
426
427 void getDataSetWatchFailure(String path) {
428 synchronized (grabTaskLock) {
429 if (workerInGrabTask) {
430
431 String taskpath = currentTask;
432 if (taskpath != null && taskpath.equals(path)) {
433 LOG.info("retrying data watch on " + path);
434 tot_wkr_get_data_retry.incrementAndGet();
435 getDataSetWatchAsync();
436 } else {
437
438
439 }
440 }
441 }
442 }
443
444
445
446
447 @Override
448 public void nodeDataChanged(String path) {
449
450
451 synchronized (grabTaskLock) {
452 if (workerInGrabTask) {
453
454 String taskpath = currentTask;
455 if (taskpath!= null && taskpath.equals(path)) {
456 getDataSetWatchAsync();
457 }
458 }
459 }
460 }
461
462
463 private List<String> getTaskList() {
464 for (int i = 0; i < zkretries; i++) {
465 try {
466 return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
467 this.watcher.splitLogZNode));
468 } catch (KeeperException e) {
469 LOG.warn("Could not get children of znode " +
470 this.watcher.splitLogZNode, e);
471 try {
472 Thread.sleep(1000);
473 } catch (InterruptedException e1) {
474 LOG.warn("Interrupted while trying to get task list ...", e1);
475 Thread.currentThread().interrupt();
476 return null;
477 }
478 }
479 }
480 LOG.warn("Tried " + zkretries + " times, still couldn't fetch " +
481 "children of " + watcher.splitLogZNode + " giving up");
482 return null;
483 }
484
485
486 @Override
487 public void nodeChildrenChanged(String path) {
488 if(path.equals(watcher.splitLogZNode)) {
489 LOG.debug("tasks arrived or departed");
490 synchronized (taskReadyLock) {
491 taskReadySeq++;
492 taskReadyLock.notify();
493 }
494 }
495 }
496
497
498
499
500
501 void stopTask() {
502 LOG.info("Sending interrupt to stop the worker thread");
503 worker.interrupt();
504 }
505
506
507
508
509
510 public void start() {
511 worker = new Thread(null, this, "SplitLogWorker-" + serverName);
512 exitWorker = false;
513 worker.start();
514 return;
515 }
516
517
518
519
520 public void stop() {
521 exitWorker = true;
522 stopTask();
523 }
524
525
526
527
528 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
529 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
530
531 @Override
532 public void processResult(int rc, String path, Object ctx, byte[] data,
533 Stat stat) {
534 tot_wkr_get_data_result.incrementAndGet();
535 if (rc != 0) {
536 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
537 getDataSetWatchFailure(path);
538 return;
539 }
540 data = watcher.getRecoverableZooKeeper().removeMetaData(data);
541 getDataSetWatchSuccess(path, data);
542 return;
543 }
544 }
545
546
547
548
549
550
551
552
553 static public interface TaskExecutor {
554 static public enum Status {
555 DONE(),
556 ERR(),
557 RESIGNED(),
558 PREEMPTED();
559 }
560 public Status exec(String name, CancelableProgressable p);
561 }
562 }