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