1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
23  
24  import java.io.IOException;
25  import java.util.Arrays;
26  import java.util.UUID;
27  import java.util.concurrent.atomic.AtomicLong;
28  
29  import junit.framework.Assert;
30  import static org.junit.Assert.*;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.*;
38  import org.apache.hadoop.hbase.master.SplitLogManager.Task;
39  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
40  import org.apache.hadoop.hbase.regionserver.TestMasterAddressManager.NodeCreationListener;
41  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
42  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
43  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
44  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
45  import org.apache.log4j.Level;
46  import org.apache.log4j.Logger;
47  import org.apache.zookeeper.CreateMode;
48  import org.apache.zookeeper.KeeperException;
49  import org.apache.zookeeper.ZooDefs.Ids;
50  import org.junit.After;
51  import org.junit.Before;
52  import org.junit.FixMethodOrder;
53  import org.junit.Test;
54  import org.junit.experimental.categories.Category;
55  import org.junit.runners.MethodSorters;
56  import org.mockito.Mockito;
57  
58  @FixMethodOrder(MethodSorters.NAME_ASCENDING)
59  @Category(MediumTests.class)
60  public class TestSplitLogManager {
61    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
62    static {
63      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
64    }
65  
66    private ZooKeeperWatcher zkw;
67    private static boolean stopped = false;
68    private SplitLogManager slm;
69    private Configuration conf;
70    private int to;
71    private final ServerManager sm = Mockito.mock(ServerManager.class);
72    private final MasterServices master = Mockito.mock(MasterServices.class);
73  
74    private final static HBaseTestingUtility TEST_UTIL =
75      new HBaseTestingUtility();
76  
77    static Stoppable stopper = new Stoppable() {
78      @Override
79      public void stop(String why) {
80        stopped = true;
81      }
82  
83      @Override
84      public boolean isStopped() {
85        return stopped;
86      }
87  
88    };
89  
90    @Before
91    public void setup() throws Exception {
92      TEST_UTIL.startMiniZKCluster();
93      conf = TEST_UTIL.getConfiguration();
94      zkw = new ZooKeeperWatcher(conf, "split-log-manager-tests", null);
95      ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
96      ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
97      assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
98      LOG.debug(zkw.baseZNode + " created");
99      ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
100     assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
101     LOG.debug(zkw.splitLogZNode + " created");
102 
103     stopped = false;
104     resetCounters();
105     to = 6000;
106     conf.setInt("hbase.splitlog.manager.timeout", to);
107     conf.setInt("hbase.splitlog.manager.unassigned.timeout", 2 * to);
108     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
109     to = to + 4 * 100;
110 
111     // By default, we let the test manage the error as before, so the server
112     // does not appear as dead from the master point of view, only from the split log pov.
113     Mockito.when(sm.isServerOnline(Mockito.any(ServerName.class))).thenReturn(true);
114     Mockito.when(master.getServerManager()).thenReturn(sm);
115   }
116 
117   @After
118   public void teardown() throws IOException, KeeperException {
119     stopper.stop("");
120     slm.stop();
121     TEST_UTIL.shutdownMiniZKCluster();
122   }
123 
124   private interface Expr {
125     public long eval();
126   }
127 
128   private void waitForCounter(final AtomicLong ctr, long oldval, long newval,
129       long timems) {
130     Expr e = new Expr() {
131       public long eval() {
132         return ctr.get();
133       }
134     };
135     waitForCounter(e, oldval, newval, timems);
136     return;
137   }
138 
139   private void waitForCounter(Expr e, long oldval, long newval,
140       long timems) {
141     long curt = System.currentTimeMillis();
142     long endt = curt + timems;
143     while (curt < endt) {
144       if (e.eval() == oldval) {
145         try {
146           Thread.sleep(10);
147         } catch (InterruptedException eintr) {
148         }
149         curt = System.currentTimeMillis();
150       } else {
151         assertEquals(newval, e.eval());
152         return;
153       }
154     }
155     assertTrue(false);
156   }
157 
158   private String submitTaskAndWait(TaskBatch batch, String name)
159   throws KeeperException, InterruptedException {
160     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name);
161     NodeCreationListener listener = new NodeCreationListener(zkw, tasknode);
162     zkw.registerListener(listener);
163     ZKUtil.watchAndCheckExists(zkw, tasknode);
164 
165     slm.enqueueSplitTask(name, batch);
166     assertEquals(1, batch.installed);
167     assertTrue(slm.findOrCreateOrphanTask(tasknode).batch == batch);
168     assertEquals(1L, tot_mgr_node_create_queued.get());
169 
170     LOG.debug("waiting for task node creation");
171     listener.waitForCreation();
172     LOG.debug("task created");
173     return tasknode;
174   }
175 
176   /**
177    * Test whether the splitlog correctly creates a task in zookeeper
178    * @throws Exception
179    */
180   @Test
181   public void testTaskCreation() throws Exception {
182     LOG.info("TestTaskCreation - test the creation of a task in zk");
183 
184     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
185     slm.finishInitialization();
186     TaskBatch batch = new TaskBatch();
187 
188     String tasknode = submitTaskAndWait(batch, "foo/1");
189 
190     byte[] data = ZKUtil.getData(zkw, tasknode);
191     LOG.info("Task node created " + new String(data));
192     assertTrue(TaskState.TASK_UNASSIGNED.equals(data, "dummy-master"));
193   }
194 
195   @Test
196   public void testOrphanTaskAcquisition() throws Exception {
197     LOG.info("TestOrphanTaskAcquisition");
198 
199     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
200     zkw.getRecoverableZooKeeper().create(tasknode,
201         TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
202         CreateMode.PERSISTENT);
203 
204     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
205     slm.finishInitialization();
206     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
207     Task task = slm.findOrCreateOrphanTask(tasknode);
208     assertTrue(task.isOrphan());
209     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
210     assertFalse(task.isUnassigned());
211     long curt = System.currentTimeMillis();
212     assertTrue((task.last_update <= curt) &&
213         (task.last_update > (curt - 1000)));
214     LOG.info("waiting for manager to resubmit the orphan task");
215     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
216     assertTrue(task.isUnassigned());
217     waitForCounter(tot_mgr_rescan, 0, 1, to + to/2);
218   }
219 
220   @Test
221   public void testUnassignedOrphan() throws Exception {
222     LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" +
223         " startup");
224     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
225     //create an unassigned orphan task
226     zkw.getRecoverableZooKeeper().create(tasknode,
227         TaskState.TASK_UNASSIGNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
228         CreateMode.PERSISTENT);
229     int version = ZKUtil.checkExists(zkw, tasknode);
230 
231     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
232     slm.finishInitialization();
233     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
234     Task task = slm.findOrCreateOrphanTask(tasknode);
235     assertTrue(task.isOrphan());
236     assertTrue(task.isUnassigned());
237     // wait for RESCAN node to be created
238     waitForCounter(tot_mgr_rescan, 0, 1, to/2);
239     Task task2 = slm.findOrCreateOrphanTask(tasknode);
240     assertTrue(task == task2);
241     LOG.debug("task = " + task);
242     assertEquals(1L, tot_mgr_resubmit.get());
243     assertEquals(1, task.incarnation);
244     assertEquals(0, task.unforcedResubmits);
245     assertTrue(task.isOrphan());
246     assertTrue(task.isUnassigned());
247     assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
248   }
249 
250   @Test
251   public void testMultipleResubmits() throws Exception {
252     LOG.info("TestMultipleResbmits - no indefinite resubmissions");
253 
254     conf.setInt("hbase.splitlog.max.resubmit", 2);
255     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
256     slm.finishInitialization();
257     TaskBatch batch = new TaskBatch();
258 
259     String tasknode = submitTaskAndWait(batch, "foo/1");
260     int version = ZKUtil.checkExists(zkw, tasknode);
261 
262     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
263     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
264     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
265     int version1 = ZKUtil.checkExists(zkw, tasknode);
266     assertTrue(version1 > version);
267     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker2"));
268     waitForCounter(tot_mgr_heartbeat, 1, 2, to/2);
269     waitForCounter(tot_mgr_resubmit, 1, 2, to + to/2);
270     int version2 = ZKUtil.checkExists(zkw, tasknode);
271     assertTrue(version2 > version1);
272     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker3"));
273     waitForCounter(tot_mgr_heartbeat, 1, 2, to/2);
274     waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to/2);
275     Thread.sleep(to + to/2);
276     assertEquals(2L, tot_mgr_resubmit.get());
277   }
278 
279   @Test
280   public void testRescanCleanup() throws Exception {
281     LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");
282 
283     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
284     slm.finishInitialization();
285     TaskBatch batch = new TaskBatch();
286 
287     String tasknode = submitTaskAndWait(batch, "foo/1");
288     int version = ZKUtil.checkExists(zkw, tasknode);
289 
290     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
291     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
292     waitForCounter(new Expr() {
293       @Override
294       public long eval() {
295         return (tot_mgr_resubmit.get() + tot_mgr_resubmit_failed.get());
296       }
297     }, 0, 1, 5*60000); // wait long enough
298     Assert
299         .assertEquals("Could not run test. Lost ZK connection?", 0, tot_mgr_resubmit_failed.get());
300     int version1 = ZKUtil.checkExists(zkw, tasknode);
301     assertTrue(version1 > version);
302     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
303     assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"), taskstate));
304 
305     waitForCounter(tot_mgr_rescan_deleted, 0, 1, to / 2);
306   }
307 
308   @Test
309   public void testTaskDone() throws Exception {
310     LOG.info("TestTaskDone - cleanup task node once in DONE state");
311 
312     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
313     slm.finishInitialization();
314     TaskBatch batch = new TaskBatch();
315     String tasknode = submitTaskAndWait(batch, "foo/1");
316     ZKUtil.setData(zkw, tasknode, TaskState.TASK_DONE.get("worker"));
317     synchronized (batch) {
318       while (batch.installed != batch.done) {
319         batch.wait();
320       }
321     }
322     waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
323     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
324   }
325 
326   @Test
327   public void testTaskErr() throws Exception {
328     LOG.info("TestTaskErr - cleanup task node once in ERR state");
329 
330     conf.setInt("hbase.splitlog.max.resubmit", 0);
331     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
332     slm.finishInitialization();
333     TaskBatch batch = new TaskBatch();
334 
335     String tasknode = submitTaskAndWait(batch, "foo/1");
336     ZKUtil.setData(zkw, tasknode, TaskState.TASK_ERR.get("worker"));
337     synchronized (batch) {
338       while (batch.installed != batch.error) {
339         batch.wait();
340       }
341     }
342     waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
343     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
344     conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
345   }
346 
347   @Test
348   public void testTaskResigned() throws Exception {
349     LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
350 
351     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
352     slm.finishInitialization();
353     TaskBatch batch = new TaskBatch();
354     String tasknode = submitTaskAndWait(batch, "foo/1");
355     ZKUtil.setData(zkw, tasknode, TaskState.TASK_RESIGNED.get("worker"));
356     int version = ZKUtil.checkExists(zkw, tasknode);
357 
358     waitForCounter(tot_mgr_resubmit, 0, 1, to/2);
359     int version1 = ZKUtil.checkExists(zkw, tasknode);
360     assertTrue(version1 > version);
361 
362     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
363     assertTrue(Arrays.equals(taskstate,
364         TaskState.TASK_UNASSIGNED.get("dummy-master")));
365   }
366 
367   @Test
368   public void testUnassignedTimeout() throws Exception {
369     LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" +
370         " resubmit");
371 
372     // create an orphan task in OWNED state
373     String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
374     zkw.getRecoverableZooKeeper().create(tasknode1,
375         TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
376         CreateMode.PERSISTENT);
377 
378     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
379     slm.finishInitialization();
380     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
381 
382 
383     // submit another task which will stay in unassigned mode
384     TaskBatch batch = new TaskBatch();
385     submitTaskAndWait(batch, "foo/1");
386 
387     // keep updating the orphan owned node every to/2 seconds
388     for (int i = 0; i < (3 * to)/100; i++) {
389       Thread.sleep(100);
390       ZKUtil.setData(zkw, tasknode1,
391           TaskState.TASK_OWNED.get("dummy-worker"));
392     }
393 
394     // since we have stopped heartbeating the owned node therefore it should
395     // get resubmitted
396     LOG.info("waiting for manager to resubmit the orphan task");
397     waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
398 
399     // now all the nodes are unassigned. manager should post another rescan
400     waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + to/2);
401   }
402 
403   @Test
404   public void testDeadWorker() throws Exception {
405     LOG.info("testDeadWorker");
406 
407     conf.setLong("hbase.splitlog.max.resubmit", 0);
408     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
409     slm.finishInitialization();
410     TaskBatch batch = new TaskBatch();
411 
412     String tasknode = submitTaskAndWait(batch, "foo/1");
413     int version = ZKUtil.checkExists(zkw, tasknode);
414 
415     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
416     waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
417     slm.handleDeadWorker("worker1");
418     waitForCounter(tot_mgr_resubmit, 0, 1, to/2);
419     waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, to + to/2);
420 
421     int version1 = ZKUtil.checkExists(zkw, tasknode);
422     assertTrue(version1 > version);
423     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
424     assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
425         taskstate));
426     return;
427   }
428 
429   @Test
430   public void testEmptyLogDir() throws Exception {
431     LOG.info("testEmptyLogDir");
432     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
433     slm.finishInitialization();
434     FileSystem fs = TEST_UTIL.getTestFileSystem();
435     Path emptyLogDirPath = new Path(fs.getWorkingDirectory(),
436         UUID.randomUUID().toString());
437     fs.mkdirs(emptyLogDirPath);
438     slm.splitLogDistributed(emptyLogDirPath);
439     assertFalse(fs.exists(emptyLogDirPath));
440   }
441 
442   @Test(timeout=45000)
443   public void testVanishingTaskZNode() throws Exception {
444     LOG.info("testVanishingTaskZNode");
445     conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
446     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 1000);
447     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
448     slm.finishInitialization();
449     FileSystem fs = TEST_UTIL.getTestFileSystem();
450     final Path logDir = new Path(fs.getWorkingDirectory(),
451         UUID.randomUUID().toString());
452     fs.mkdirs(logDir);
453     Thread thread = null;
454     try {
455       Path logFile = new Path(logDir, UUID.randomUUID().toString());
456       fs.createNewFile(logFile);
457       thread = new Thread() {
458         public void run() {
459           try {
460             // this call will block because there are no SplitLogWorkers,
461             // until the task znode is deleted below. Then the call will
462             // complete successfully, assuming the log is split.
463             slm.splitLogDistributed(logDir);
464           } catch (Exception e) {
465             LOG.warn("splitLogDistributed failed", e);
466           }
467         }
468       };
469       thread.start();
470       waitForCounter(tot_mgr_node_create_result, 0, 1, 10000);
471       String znode = ZKSplitLog.getEncodedNodeName(zkw, logFile.toString());
472       // remove the task znode, to finish the distributed log splitting
473       ZKUtil.deleteNode(zkw, znode);
474       waitForCounter(tot_mgr_get_data_nonode, 0, 1, 30000);
475       waitForCounter(tot_mgr_log_split_batch_success, 0, 1, to/2);
476       assertTrue(fs.exists(logFile));
477     } finally {
478       if (thread != null) {
479         // interrupt the thread in case the test fails in the middle.
480         // it has no effect if the thread is already terminated.
481         thread.interrupt();
482       }
483       fs.delete(logDir, true);
484     }
485   }
486 
487   @Test
488   public void testWorkerCrash() throws Exception {
489     conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
490     slm = new SplitLogManager(zkw, conf, stopper, master, "dummy-master", null);
491     slm.finishInitialization();
492     TaskBatch batch = new TaskBatch();
493 
494     String tasknode = submitTaskAndWait(batch, "foo/1");
495     final ServerName worker1 = new ServerName("worker1,1,1");
496 
497     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get(worker1.getServerName()));
498     if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to / 2);
499 
500     // Not yet resubmitted.
501     Assert.assertEquals(0, tot_mgr_resubmit.get());
502 
503     // This server becomes dead
504     Mockito.when(sm.isServerOnline(worker1)).thenReturn(false);
505 
506     Thread.sleep(1300); // The timeout checker is done every 1000 ms (hardcoded).
507 
508     // It has been resubmitted
509     Assert.assertEquals(1, tot_mgr_resubmit.get());
510   }
511 
512   @org.junit.Rule
513   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
514     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
515 }
516