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.List;
27  import java.util.concurrent.atomic.AtomicLong;
28  
29  import static org.junit.Assert.*;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.HBaseTestingUtility;
36  import org.apache.hadoop.hbase.Stoppable;
37  import org.apache.hadoop.hbase.master.SplitLogManager.Task;
38  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
39  import org.apache.hadoop.hbase.regionserver.TestMasterAddressManager.NodeCreationListener;
40  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
41  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
42  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
43  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
44  import org.apache.log4j.Level;
45  import org.apache.log4j.Logger;
46  import org.apache.zookeeper.CreateMode;
47  import org.apache.zookeeper.KeeperException;
48  import org.apache.zookeeper.ZooDefs.Ids;
49  import org.junit.After;
50  import org.junit.AfterClass;
51  import org.junit.Before;
52  import org.junit.BeforeClass;
53  import org.junit.Test;
54  
55  
56  public class TestSplitLogManager {
57    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
58    static {
59      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
60    }
61  
62    private ZooKeeperWatcher zkw;
63    private static boolean stopped = false;
64    private SplitLogManager slm;
65    private Configuration conf;
66  
67    private final static HBaseTestingUtility TEST_UTIL =
68      new HBaseTestingUtility();
69  
70    static Stoppable stopper = new Stoppable() {
71      @Override
72      public void stop(String why) {
73        stopped = true;
74      }
75  
76      @Override
77      public boolean isStopped() {
78        return stopped;
79      }
80  
81    };
82  
83    @BeforeClass
84    public static void setUpBeforeClass() throws Exception {
85    }
86  
87    @AfterClass
88    public static void tearDownAfterClass() throws Exception {
89    }
90  
91    @Before
92    public void setup() throws Exception {
93      TEST_UTIL.startMiniZKCluster();
94      conf = TEST_UTIL.getConfiguration();
95      zkw = new ZooKeeperWatcher(conf, "split-log-manager-tests", null);
96      ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
97      ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
98      assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
99      LOG.debug(zkw.baseZNode + " created");
100     ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
101     assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
102     LOG.debug(zkw.splitLogZNode + " created");
103 
104     stopped = false;
105     resetCounters();
106   }
107 
108   @After
109   public void teardown() throws IOException, KeeperException {
110     stopper.stop("");
111     slm.stop();
112     TEST_UTIL.shutdownMiniZKCluster();
113   }
114 
115   private void waitForCounter(AtomicLong ctr, long oldval, long newval,
116       long timems) {
117     long curt = System.currentTimeMillis();
118     long endt = curt + timems;
119     while (curt < endt) {
120       if (ctr.get() == oldval) {
121         try {
122           Thread.sleep(10);
123         } catch (InterruptedException e) {
124         }
125         curt = System.currentTimeMillis();
126       } else {
127         assertEquals(newval, ctr.get());
128         return;
129       }
130     }
131     assertTrue(false);
132   }
133 
134   private String submitTaskAndWait(TaskBatch batch, String name)
135   throws KeeperException, InterruptedException {
136     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name);
137     NodeCreationListener listener = new NodeCreationListener(zkw, tasknode);
138     zkw.registerListener(listener);
139     ZKUtil.watchAndCheckExists(zkw, tasknode);
140 
141     slm.installTask(name, batch);
142     assertEquals(1, batch.installed);
143     assertTrue(slm.findOrCreateOrphanTask(tasknode).batch == batch);
144     assertEquals(1L, tot_mgr_node_create_queued.get());
145 
146     LOG.debug("waiting for task node creation");
147     listener.waitForCreation();
148     LOG.debug("task created");
149     return tasknode;
150   }
151 
152   /**
153    * Test whether the splitlog correctly creates a task in zookeeper
154    * @throws Exception
155    */
156   @Test
157   public void testTaskCreation() throws Exception {
158     LOG.info("TestTaskCreation - test the creation of a task in zk");
159 
160     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
161     slm.finishInitialization();
162     TaskBatch batch = new TaskBatch();
163 
164     String tasknode = submitTaskAndWait(batch, "foo/1");
165 
166     byte[] data = ZKUtil.getData(zkw, tasknode);
167     LOG.info("Task node created " + new String(data));
168     assertTrue(TaskState.TASK_UNASSIGNED.equals(data, "dummy-master"));
169   }
170 
171   @Test
172   public void testOrphanTaskAcquisition() throws Exception {
173     LOG.info("TestOrphanTaskAcquisition");
174 
175     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
176     zkw.getRecoverableZooKeeper().create(tasknode,
177         TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
178         CreateMode.PERSISTENT);
179 
180     int to = 1000;
181     conf.setInt("hbase.splitlog.manager.timeout", to);
182     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
183     to = to + 2 * 100;
184 
185 
186     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
187     slm.finishInitialization();
188     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
189     Task task = slm.findOrCreateOrphanTask(tasknode);
190     assertTrue(task.isOrphan());
191     waitForCounter(tot_mgr_heartbeat, 0, 1, 100);
192     assertFalse(task.isUnassigned());
193     long curt = System.currentTimeMillis();
194     assertTrue((task.last_update <= curt) &&
195         (task.last_update > (curt - 1000)));
196     LOG.info("waiting for manager to resubmit the orphan task");
197     waitForCounter(tot_mgr_resubmit, 0, 1, to + 100);
198     assertTrue(task.isUnassigned());
199     waitForCounter(tot_mgr_rescan, 0, 1, to + 100);
200   }
201 
202   @Test
203   public void testUnassignedOrphan() throws Exception {
204     LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" +
205         " startup");
206     String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
207     //create an unassigned orphan task
208     zkw.getRecoverableZooKeeper().create(tasknode,
209         TaskState.TASK_UNASSIGNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
210         CreateMode.PERSISTENT);
211     int version = ZKUtil.checkExists(zkw, tasknode);
212 
213     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
214     slm.finishInitialization();
215     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
216     Task task = slm.findOrCreateOrphanTask(tasknode);
217     assertTrue(task.isOrphan());
218     assertTrue(task.isUnassigned());
219     // wait for RESCAN node to be created
220     waitForCounter(tot_mgr_rescan, 0, 1, 500);
221     Task task2 = slm.findOrCreateOrphanTask(tasknode);
222     assertTrue(task == task2);
223     LOG.debug("task = " + task);
224     assertEquals(1L, tot_mgr_resubmit.get());
225     assertEquals(1, task.incarnation);
226     assertEquals(0, task.unforcedResubmits);
227     assertTrue(task.isOrphan());
228     assertTrue(task.isUnassigned());
229     assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
230   }
231 
232   @Test
233   public void testMultipleResubmits() throws Exception {
234     LOG.info("TestMultipleResbmits - no indefinite resubmissions");
235 
236     int to = 1000;
237     conf.setInt("hbase.splitlog.manager.timeout", to);
238     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
239     to = to + 2 * 100;
240 
241     conf.setInt("hbase.splitlog.max.resubmit", 2);
242     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
243     slm.finishInitialization();
244     TaskBatch batch = new TaskBatch();
245 
246     String tasknode = submitTaskAndWait(batch, "foo/1");
247     int version = ZKUtil.checkExists(zkw, tasknode);
248 
249     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
250     waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
251     waitForCounter(tot_mgr_resubmit, 0, 1, to + 100);
252     int version1 = ZKUtil.checkExists(zkw, tasknode);
253     assertTrue(version1 > version);
254     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker2"));
255     waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
256     waitForCounter(tot_mgr_resubmit, 1, 2, to + 100);
257     int version2 = ZKUtil.checkExists(zkw, tasknode);
258     assertTrue(version2 > version1);
259     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker3"));
260     waitForCounter(tot_mgr_heartbeat, 1, 2, 1000);
261     waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + 100);
262     Thread.sleep(to + 100);
263     assertEquals(2L, tot_mgr_resubmit.get());
264   }
265 
266   @Test
267   public void testRescanCleanup() throws Exception {
268     LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");
269 
270     int to = 1000;
271     conf.setInt("hbase.splitlog.manager.timeout", to);
272     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
273     to = to + 2 * 100;
274     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
275     slm.finishInitialization();
276     TaskBatch batch = new TaskBatch();
277 
278     String tasknode = submitTaskAndWait(batch, "foo/1");
279     int version = ZKUtil.checkExists(zkw, tasknode);
280 
281     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
282     waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
283     waitForCounter(tot_mgr_resubmit, 0, 1, to + 100);
284     int version1 = ZKUtil.checkExists(zkw, tasknode);
285     assertTrue(version1 > version);
286     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
287     assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
288         taskstate));
289 
290     waitForCounter(tot_mgr_rescan_deleted, 0, 1, 1000);
291 
292     return;
293   }
294 
295   @Test
296   public void testTaskDone() throws Exception {
297     LOG.info("TestTaskDone - cleanup task node once in DONE state");
298 
299     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
300     slm.finishInitialization();
301     TaskBatch batch = new TaskBatch();
302     String tasknode = submitTaskAndWait(batch, "foo/1");
303     ZKUtil.setData(zkw, tasknode, TaskState.TASK_DONE.get("worker"));
304     synchronized (batch) {
305       while (batch.installed != batch.done) {
306         batch.wait();
307       }
308     }
309     waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
310     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
311   }
312 
313   @Test
314   public void testTaskErr() throws Exception {
315     LOG.info("TestTaskErr - cleanup task node once in ERR state");
316 
317     conf.setInt("hbase.splitlog.max.resubmit", 0);
318     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
319     slm.finishInitialization();
320     TaskBatch batch = new TaskBatch();
321 
322     String tasknode = submitTaskAndWait(batch, "foo/1");
323     ZKUtil.setData(zkw, tasknode, TaskState.TASK_ERR.get("worker"));
324     synchronized (batch) {
325       while (batch.installed != batch.error) {
326         batch.wait();
327       }
328     }
329     waitForCounter(tot_mgr_task_deleted, 0, 1, 1000);
330     assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
331     conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLog.DEFAULT_MAX_RESUBMIT);
332   }
333 
334   @Test
335   public void testTaskResigned() throws Exception {
336     LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
337 
338     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
339     slm.finishInitialization();
340     TaskBatch batch = new TaskBatch();
341     String tasknode = submitTaskAndWait(batch, "foo/1");
342     ZKUtil.setData(zkw, tasknode, TaskState.TASK_RESIGNED.get("worker"));
343     int version = ZKUtil.checkExists(zkw, tasknode);
344 
345     waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
346     int version1 = ZKUtil.checkExists(zkw, tasknode);
347     assertTrue(version1 > version);
348 
349     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
350     assertTrue(Arrays.equals(taskstate,
351         TaskState.TASK_UNASSIGNED.get("dummy-master")));
352   }
353 
354   @Test
355   public void testUnassignedTimeout() throws Exception {
356     LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" +
357         " resubmit");
358 
359     // create an orphan task in OWNED state
360     String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
361     zkw.getRecoverableZooKeeper().create(tasknode1,
362         TaskState.TASK_OWNED.get("dummy-worker"), Ids.OPEN_ACL_UNSAFE,
363         CreateMode.PERSISTENT);
364 
365     int to = 1000;
366     conf.setInt("hbase.splitlog.manager.timeout", to);
367     conf.setInt("hbase.splitlog.manager.unassigned.timeout", 2 * to);
368     conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
369 
370 
371     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
372     slm.finishInitialization();
373     waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, 100);
374 
375 
376     // submit another task which will stay in unassigned mode
377     TaskBatch batch = new TaskBatch();
378     submitTaskAndWait(batch, "foo/1");
379 
380     // keep updating the orphan owned node every to/2 seconds
381     for (int i = 0; i < (3 * to)/100; i++) {
382       Thread.sleep(100);
383       ZKUtil.setData(zkw, tasknode1,
384           TaskState.TASK_OWNED.get("dummy-worker"));
385     }
386 
387     // since we have stopped heartbeating the owned node therefore it should
388     // get resubmitted
389     LOG.info("waiting for manager to resubmit the orphan task");
390     waitForCounter(tot_mgr_resubmit, 0, 1, to + 500);
391 
392     // now all the nodes are unassigned. manager should post another rescan
393     waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + 500);
394   }
395 
396   @Test
397   public void testDeadWorker() throws Exception {
398     LOG.info("testDeadWorker");
399 
400     conf.setLong("hbase.splitlog.max.resubmit", 0);
401     slm = new SplitLogManager(zkw, conf, stopper, "dummy-master", null);
402     slm.finishInitialization();
403     TaskBatch batch = new TaskBatch();
404 
405     String tasknode = submitTaskAndWait(batch, "foo/1");
406     int version = ZKUtil.checkExists(zkw, tasknode);
407 
408     ZKUtil.setData(zkw, tasknode, TaskState.TASK_OWNED.get("worker1"));
409     waitForCounter(tot_mgr_heartbeat, 0, 1, 1000);
410     slm.handleDeadWorker("worker1");
411     waitForCounter(tot_mgr_resubmit, 0, 1, 1000);
412     waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, 1000);
413 
414     int version1 = ZKUtil.checkExists(zkw, tasknode);
415     assertTrue(version1 > version);
416     byte[] taskstate = ZKUtil.getData(zkw, tasknode);
417     assertTrue(Arrays.equals(TaskState.TASK_UNASSIGNED.get("dummy-master"),
418         taskstate));
419     return;
420   }
421 }