View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master;
20  
21  import static org.apache.hadoop.hbase.SplitLogCounters.resetCounters;
22  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_heartbeat;
23  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_queued;
24  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_orphan_task_acquired;
25  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan;
26  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan_deleted;
27  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit;
28  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_dead_server_task;
29  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_failed;
30  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_force;
31  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_threshold_reached;
32  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_unassigned;
33  import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_task_deleted;
34  import static org.junit.Assert.assertEquals;
35  import static org.junit.Assert.assertFalse;
36  import static org.junit.Assert.assertTrue;
37  
38  import java.io.IOException;
39  import java.util.List;
40  import java.util.Map;
41  import java.util.UUID;
42  import java.util.concurrent.atomic.AtomicLong;
43  
44  import org.apache.commons.logging.Log;
45  import org.apache.commons.logging.LogFactory;
46  import org.apache.hadoop.conf.Configuration;
47  import org.apache.hadoop.fs.FileSystem;
48  import org.apache.hadoop.fs.Path;
49  import org.apache.hadoop.hbase.CoordinatedStateManager;
50  import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
51  import org.apache.hadoop.hbase.HBaseConfiguration;
52  import org.apache.hadoop.hbase.HBaseTestingUtility;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HRegionInfo;
55  import org.apache.hadoop.hbase.testclassification.MediumTests;
56  import org.apache.hadoop.hbase.Server;
57  import org.apache.hadoop.hbase.ServerName;
58  import org.apache.hadoop.hbase.SplitLogCounters;
59  import org.apache.hadoop.hbase.SplitLogTask;
60  import org.apache.hadoop.hbase.Stoppable;
61  import org.apache.hadoop.hbase.Waiter;
62  import org.apache.hadoop.hbase.client.ClusterConnection;
63  import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
64  import org.apache.hadoop.hbase.master.SplitLogManager.Task;
65  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
66  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
67  import org.apache.hadoop.hbase.regionserver.TestMasterAddressTracker.NodeCreationListener;
68  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
69  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
70  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
71  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
72  import org.apache.log4j.Level;
73  import org.apache.log4j.Logger;
74  import org.apache.zookeeper.CreateMode;
75  import org.apache.zookeeper.KeeperException;
76  import org.apache.zookeeper.ZooDefs.Ids;
77  import org.junit.After;
78  import org.junit.Assert;
79  import org.junit.Before;
80  import org.junit.Test;
81  import org.junit.experimental.categories.Category;
82  import org.mockito.Mockito;
83  
84  @Category(MediumTests.class)
85  public class TestSplitLogManager {
86    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
87    private final ServerName DUMMY_MASTER = ServerName.valueOf("dummy-master,1,1");
88    private final ServerManager sm = Mockito.mock(ServerManager.class);
89    private final MasterServices master = Mockito.mock(MasterServices.class);
90  
91    static {
92      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
93    }
94  
95    private ZooKeeperWatcher zkw;
96    private DummyServer ds;
97    private static boolean stopped = false;
98    private SplitLogManager slm;
99    private Configuration conf;
100   private int to;
101   private RecoveryMode mode;
102 
103   private static HBaseTestingUtility TEST_UTIL;
104 
105   class DummyServer implements Server {
106     private ZooKeeperWatcher zkw;
107     private Configuration conf;
108     private CoordinatedStateManager cm;
109 
110     public DummyServer(ZooKeeperWatcher zkw, Configuration conf) {
111       this.zkw = zkw;
112       this.conf = conf;
113       cm = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
114       cm.initialize(this);
115     }
116 
117     @Override
118     public void abort(String why, Throwable e) {
119     }
120 
121     @Override
122     public boolean isAborted() {
123       return false;
124     }
125 
126     @Override
127     public void stop(String why) {
128     }
129 
130     @Override
131     public boolean isStopped() {
132       return false;
133     }
134 
135     @Override
136     public Configuration getConfiguration() {
137       return conf;
138     }
139 
140     @Override
141     public ZooKeeperWatcher getZooKeeper() {
142       return zkw;
143     }
144 
145     @Override
146     public ServerName getServerName() {
147       return null;
148     }
149 
150     @Override
151     public CoordinatedStateManager getCoordinatedStateManager() {
152       return cm;
153     }
154 
155     @Override
156     public ClusterConnection getConnection() {
157       return null;
158     }
159 
160     @Override
161     public MetaTableLocator getMetaTableLocator() {
162       return null;
163     }
164   }
165 
166   static Stoppable stopper = new Stoppable() {
167     @Override
168     public void stop(String why) {
169       stopped = true;
170     }
171 
172     @Override
173     public boolean isStopped() {
174       return stopped;
175     }
176   };
177 
178   @Before
179   public void setup() throws Exception {
180     TEST_UTIL = new HBaseTestingUtility();
181     TEST_UTIL.startMiniZKCluster();
182     conf = TEST_UTIL.getConfiguration();
183     // Use a different ZK wrapper instance for each tests.
184     zkw =
185         new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
186     ds = new DummyServer(zkw, conf);
187 
188     ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
189     ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
190     assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
191     LOG.debug(zkw.baseZNode + " created");
192     ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
193     assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
194     LOG.debug(zkw.splitLogZNode + " created");
195 
196     stopped = false;
197     resetCounters();
198 
199     // By default, we let the test manage the error as before, so the server
200     // does not appear as dead from the master point of view, only from the split log pov.
201     Mockito.when(sm.isServerOnline(Mockito.any(ServerName.class))).thenReturn(true);
202     Mockito.when(master.getServerManager()).thenReturn(sm);
203 
204     to = 12000;
205     conf.setInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT, to);
206     conf.setInt("hbase.splitlog.manager.unassigned.timeout", 2 * to);
207 
208     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
209     to = to + 4 * 100;
210 
211     this.mode =
212         (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? RecoveryMode.LOG_REPLAY
213             : RecoveryMode.LOG_SPLITTING);
214   }
215 
216   @After
217   public void teardown() throws IOException, KeeperException {
218     stopper.stop("");
219     if (slm != null) slm.stop();
220     TEST_UTIL.shutdownMiniZKCluster();
221   }
222 
223   private interface Expr {
224     long eval();
225   }
226 
227   private void waitForCounter(final AtomicLong ctr, long oldval, long newval, long timems)
228       throws Exception {
229     Expr e = new Expr() {
230       @Override
231       public long eval() {
232         return ctr.get();
233       }
234     };
235     waitForCounter(e, oldval, newval, timems);
236     return;
237   }
238 
239   private void waitForCounter(final Expr e, final long oldval, long newval, long timems)
240       throws Exception {
241 
242     TEST_UTIL.waitFor(timems, 10, new Waiter.Predicate<Exception>() {
243       @Override
244       public boolean evaluate() throws Exception {
245         return (e.eval() != oldval);
246       }
247     });
248 
249     assertEquals(newval, e.eval());
250   }
251 
252   private String submitTaskAndWait(TaskBatch batch, String name) throws KeeperException,
253       InterruptedException {
254     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name);
255     NodeCreationListener listener = new NodeCreationListener(zkw, tasknode);
256     zkw.registerListener(listener);
257     ZKUtil.watchAndCheckExists(zkw, tasknode);
258 
259     slm.enqueueSplitTask(name, batch);
260     assertEquals(1, batch.installed);
261     assertTrue(slm.findOrCreateOrphanTask(tasknode).batch == batch);
262     assertEquals(1L, tot_mgr_node_create_queued.get());
263 
264     LOG.debug("waiting for task node creation");
265     listener.waitForCreation();
266     LOG.debug("task created");
267     return tasknode;
268   }
269 
270   /**
271    * Test whether the splitlog correctly creates a task in zookeeper
272    * @throws Exception
273    */
274   @Test (timeout=180000)
275   public void testTaskCreation() throws Exception {
276 
277     LOG.info("TestTaskCreation - test the creation of a task in zk");
278     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
279     TaskBatch batch = new TaskBatch();
280 
281     String tasknode = submitTaskAndWait(batch, "foo/1");
282 
283     byte[] data = ZKUtil.getData(zkw, tasknode);
284     SplitLogTask slt = SplitLogTask.parseFrom(data);
285     LOG.info("Task node created " + slt.toString());
286     assertTrue(slt.isUnassigned(DUMMY_MASTER));
287   }
288 
289   @Test (timeout=180000)
290   public void testOrphanTaskAcquisition() throws Exception {
291     LOG.info("TestOrphanTaskAcquisition");
292 
293     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
294     SplitLogTask slt = new SplitLogTask.Owned(DUMMY_MASTER, this.mode);
295     zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
296         CreateMode.PERSISTENT);
297 
298     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
299     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
300     Task task = slm.findOrCreateOrphanTask(tasknode);
301     assertTrue(task.isOrphan());
302     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
303     assertFalse(task.isUnassigned());
304     long curt = System.currentTimeMillis();
305     assertTrue((task.last_update <= curt) &&
306         (task.last_update > (curt - 1000)));
307     LOG.info("waiting for manager to resubmit the orphan task");
308     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
309     assertTrue(task.isUnassigned());
310     waitForCounter(tot_mgr_rescan, 0, 1, to + to/2);
311   }
312 
313   @Test (timeout=180000)
314   public void testUnassignedOrphan() throws Exception {
315     LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" +
316         " startup");
317     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
318     //create an unassigned orphan task
319     SplitLogTask slt = new SplitLogTask.Unassigned(DUMMY_MASTER, this.mode);
320     zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
321         CreateMode.PERSISTENT);
322     int version = ZKUtil.checkExists(zkw, tasknode);
323 
324     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
325     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
326     Task task = slm.findOrCreateOrphanTask(tasknode);
327     assertTrue(task.isOrphan());
328     assertTrue(task.isUnassigned());
329     // wait for RESCAN node to be created
330     waitForCounter(tot_mgr_rescan, 0, 1, to/2);
331     Task task2 = slm.findOrCreateOrphanTask(tasknode);
332     assertTrue(task == task2);
333     LOG.debug("task = " + task);
334     assertEquals(1L, tot_mgr_resubmit.get());
335     assertEquals(1, task.incarnation);
336     assertEquals(0, task.unforcedResubmits.get());
337     assertTrue(task.isOrphan());
338     assertTrue(task.isUnassigned());
339     assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
340   }
341 
342   @Test (timeout=180000)
343   public void testMultipleResubmits() throws Exception {
344     LOG.info("TestMultipleResbmits - no indefinite resubmissions");
345     conf.setInt("hbase.splitlog.max.resubmit", 2);
346     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
347     TaskBatch batch = new TaskBatch();
348 
349     String tasknode = submitTaskAndWait(batch, "foo/1");
350     int version = ZKUtil.checkExists(zkw, tasknode);
351     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
352     final ServerName worker2 = ServerName.valueOf("worker2,1,1");
353     final ServerName worker3 = ServerName.valueOf("worker3,1,1");
354     SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
355     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
356     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
357     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
358     int version1 = ZKUtil.checkExists(zkw, tasknode);
359     assertTrue(version1 > version);
360     slt = new SplitLogTask.Owned(worker2, this.mode);
361     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
362     waitForCounter(tot_mgr_heartbeat, 1, 2, to/2);
363     waitForCounter(tot_mgr_resubmit, 1, 2, to + to/2);
364     int version2 = ZKUtil.checkExists(zkw, tasknode);
365     assertTrue(version2 > version1);
366     slt = new SplitLogTask.Owned(worker3, this.mode);
367     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
368     waitForCounter(tot_mgr_heartbeat, 2, 3, to/2);
369     waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to/2);
370     Thread.sleep(to + to/2);
371     assertEquals(2L, tot_mgr_resubmit.get() - tot_mgr_resubmit_force.get());
372   }
373 
374   @Test (timeout=180000)
375   public void testRescanCleanup() throws Exception {
376     LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");
377 
378     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
379     TaskBatch batch = new TaskBatch();
380 
381     String tasknode = submitTaskAndWait(batch, "foo/1");
382     int version = ZKUtil.checkExists(zkw, tasknode);
383     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
384     SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
385     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
386     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
387     waitForCounter(new Expr() {
388       @Override
389       public long eval() {
390         return (tot_mgr_resubmit.get() + tot_mgr_resubmit_failed.get());
391       }
392     }, 0, 1, 5*60000); // wait long enough
393     Assert.assertEquals("Could not run test. Lost ZK connection?", 0, tot_mgr_resubmit_failed.get());
394     int version1 = ZKUtil.checkExists(zkw, tasknode);
395     assertTrue(version1 > version);
396     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
397     slt = SplitLogTask.parseFrom(taskstate);
398     assertTrue(slt.isUnassigned(DUMMY_MASTER));
399 
400     waitForCounter(tot_mgr_rescan_deleted, 0, 1, to/2);
401   }
402 
403   @Test (timeout=180000)
404   public void testTaskDone() throws Exception {
405     LOG.info("TestTaskDone - cleanup task node once in DONE state");
406 
407     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
408     TaskBatch batch = new TaskBatch();
409     String tasknode = submitTaskAndWait(batch, "foo/1");
410     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
411     SplitLogTask slt = new SplitLogTask.Done(worker1, this.mode);
412     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
413     synchronized (batch) {
414       while (batch.installed != batch.done) {
415         batch.wait();
416       }
417     }
418     waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
419     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
420   }
421 
422   @Test (timeout=180000)
423   public void testTaskErr() throws Exception {
424     LOG.info("TestTaskErr - cleanup task node once in ERR state");
425 
426     conf.setInt("hbase.splitlog.max.resubmit", 0);
427     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
428     TaskBatch batch = new TaskBatch();
429 
430     String tasknode = submitTaskAndWait(batch, "foo/1");
431     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
432     SplitLogTask slt = new SplitLogTask.Err(worker1, this.mode);
433     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
434 
435     synchronized (batch) {
436       while (batch.installed != batch.error) {
437         batch.wait();
438       }
439     }
440     waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
441     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
442     conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLogManagerCoordination.DEFAULT_MAX_RESUBMIT);
443   }
444 
445   @Test (timeout=180000)
446   public void testTaskResigned() throws Exception {
447     LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
448     assertEquals(tot_mgr_resubmit.get(), 0);
449     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
450     assertEquals(tot_mgr_resubmit.get(), 0);
451     TaskBatch batch = new TaskBatch();
452     String tasknode = submitTaskAndWait(batch, "foo/1");
453     assertEquals(tot_mgr_resubmit.get(), 0);
454     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
455     assertEquals(tot_mgr_resubmit.get(), 0);
456     SplitLogTask slt = new SplitLogTask.Resigned(worker1, this.mode);
457     assertEquals(tot_mgr_resubmit.get(), 0);
458     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
459     int version = ZKUtil.checkExists(zkw, tasknode);
460     // Could be small race here.
461     if (tot_mgr_resubmit.get() == 0) {
462       waitForCounter(tot_mgr_resubmit, 0, 1, to/2);
463     }
464     assertEquals(tot_mgr_resubmit.get(), 1);
465 
466     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
467     slt = SplitLogTask.parseFrom(taskstate);
468     assertTrue(slt.isUnassigned(DUMMY_MASTER));
469   }
470 
471   @Test (timeout=180000)
472   public void testUnassignedTimeout() throws Exception {
473     LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" +
474         " resubmit");
475 
476     // create an orphan task in OWNED state
477     String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
478     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
479     SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
480     zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
481         CreateMode.PERSISTENT);
482 
483     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
484     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
485 
486     // submit another task which will stay in unassigned mode
487     TaskBatch batch = new TaskBatch();
488     submitTaskAndWait(batch, "foo/1");
489 
490     // keep updating the orphan owned node every to/2 seconds
491     for (int i = 0; i < (3 * to)/100; i++) {
492       Thread.sleep(100);
493       final ServerName worker2 = ServerName.valueOf("worker1,1,1");
494       slt = new SplitLogTask.Owned(worker2, this.mode);
495       ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
496     }
497 
498     // since we have stopped heartbeating the owned node therefore it should
499     // get resubmitted
500     LOG.info("waiting for manager to resubmit the orphan task");
501     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
502 
503     // now all the nodes are unassigned. manager should post another rescan
504     waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + to/2);
505   }
506 
507   @Test (timeout=180000)
508   public void testDeadWorker() throws Exception {
509     LOG.info("testDeadWorker");
510 
511     conf.setLong("hbase.splitlog.max.resubmit", 0);
512     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
513     TaskBatch batch = new TaskBatch();
514 
515     String tasknode = submitTaskAndWait(batch, "foo/1");
516     int version = ZKUtil.checkExists(zkw, tasknode);
517     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
518     SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
519     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
520     if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
521     slm.handleDeadWorker(worker1);
522     if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to+to/2);
523     if (tot_mgr_resubmit_dead_server_task.get() == 0) {
524       waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, to + to/2);
525     }
526 
527     int version1 = ZKUtil.checkExists(zkw, tasknode);
528     assertTrue(version1 > version);
529     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
530     slt = SplitLogTask.parseFrom(taskstate);
531     assertTrue(slt.isUnassigned(DUMMY_MASTER));
532     return;
533   }
534 
535   @Test (timeout=180000)
536   public void testWorkerCrash() throws Exception {
537     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
538     TaskBatch batch = new TaskBatch();
539 
540     String tasknode = submitTaskAndWait(batch, "foo/1");
541     final ServerName worker1 = ServerName.valueOf("worker1,1,1");
542 
543     SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
544     ZKUtil.setData(zkw, tasknode, slt.toByteArray());
545     if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
546 
547     // Not yet resubmitted.
548     Assert.assertEquals(0, tot_mgr_resubmit.get());
549 
550     // This server becomes dead
551     Mockito.when(sm.isServerOnline(worker1)).thenReturn(false);
552 
553     Thread.sleep(1300); // The timeout checker is done every 1000 ms (hardcoded).
554 
555     // It has been resubmitted
556     Assert.assertEquals(1, tot_mgr_resubmit.get());
557   }
558 
559   @Test (timeout=180000)
560   public void testEmptyLogDir() throws Exception {
561     LOG.info("testEmptyLogDir");
562     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
563     FileSystem fs = TEST_UTIL.getTestFileSystem();
564     Path emptyLogDirPath = new Path(fs.getWorkingDirectory(),
565         UUID.randomUUID().toString());
566     fs.mkdirs(emptyLogDirPath);
567     slm.splitLogDistributed(emptyLogDirPath);
568     assertFalse(fs.exists(emptyLogDirPath));
569   }
570 
571   @Test (timeout = 60000)
572   public void testLogFilesAreArchived() throws Exception {
573     LOG.info("testLogFilesAreArchived");
574     final SplitLogManager slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
575     FileSystem fs = TEST_UTIL.getTestFileSystem();
576     Path dir = TEST_UTIL.getDataTestDirOnTestFS("testLogFilesAreArchived");
577     conf.set(HConstants.HBASE_DIR, dir.toString());
578     Path logDirPath = new Path(dir, UUID.randomUUID().toString());
579     fs.mkdirs(logDirPath);
580     // create an empty log file
581     String logFile = ServerName.valueOf("foo", 1, 1).toString();
582     fs.create(new Path(logDirPath, logFile)).close();
583 
584     // spin up a thread mocking split done.
585     new Thread() {
586       @Override
587       public void run() {
588         boolean done = false;
589         while (!done) {
590           for (Map.Entry<String, Task> entry : slm.getTasks().entrySet()) {
591             final ServerName worker1 = ServerName.valueOf("worker1,1,1");
592             SplitLogTask slt = new SplitLogTask.Done(worker1, RecoveryMode.LOG_SPLITTING);
593             boolean encounteredZKException = false;
594             try {
595               ZKUtil.setData(zkw, entry.getKey(), slt.toByteArray());
596             } catch (KeeperException e) {
597               LOG.warn(e);
598               encounteredZKException = true;
599             }
600             if (!encounteredZKException) {
601               done = true;
602             }
603           }
604         }
605       };
606     }.start();
607 
608     slm.splitLogDistributed(logDirPath);
609 
610     assertFalse(fs.exists(logDirPath));
611   }
612 
613   /**
614    * The following test case is aiming to test the situation when distributedLogReplay is turned off
615    * and restart a cluster there should no recovery regions in ZK left.
616    * @throws Exception
617    */
618   @Test(timeout = 300000)
619   public void testRecoveryRegionRemovedFromZK() throws Exception {
620     LOG.info("testRecoveryRegionRemovedFromZK");
621     conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
622     String nodePath =
623         ZKUtil.joinZNode(zkw.recoveringRegionsZNode,
624           HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
625     ZKUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L));
626 
627     slm = new SplitLogManager(ds, conf, stopper, master, DUMMY_MASTER);
628     slm.removeStaleRecoveringRegions(null);
629 
630     List<String> recoveringRegions =
631         zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
632 
633     assertTrue("Recovery regions isn't cleaned", recoveringRegions.isEmpty());
634   }
635 
636   @Test(timeout=60000)
637   public void testGetPreviousRecoveryMode() throws Exception {
638     LOG.info("testGetPreviousRecoveryMode");
639     SplitLogCounters.resetCounters();
640     Configuration testConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
641     testConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
642     ds = new DummyServer(zkw, testConf);
643 
644     zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "testRecovery"),
645       new SplitLogTask.Unassigned(
646         ServerName.valueOf("mgr,1,1"), RecoveryMode.LOG_SPLITTING).toByteArray(),
647         Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
648 
649     slm = new SplitLogManager(ds, testConf, stopper, master, DUMMY_MASTER);
650     LOG.info("Mode1=" + slm.getRecoveryMode());
651     assertTrue(slm.isLogSplitting());
652     zkw.getRecoverableZooKeeper().delete(ZKSplitLog.getEncodedNodeName(zkw, "testRecovery"), -1);
653     LOG.info("Mode2=" + slm.getRecoveryMode());
654     slm.setRecoveryMode(false);
655     LOG.info("Mode3=" + slm.getRecoveryMode());
656     assertTrue("Mode4=" + slm.getRecoveryMode(), slm.isLogReplaying());
657   }
658 }