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.regionserver;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.resetCounters;
23  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_lost_race;
24  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_failed_to_grab_task_owned;
25  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
26  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
27  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired_rescan;
28  import static org.junit.Assert.assertEquals;
29  import static org.junit.Assert.assertTrue;
30  
31  import java.util.List;
32  import java.util.concurrent.atomic.AtomicLong;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.hbase.HBaseTestingUtility;
37  import org.apache.hadoop.hbase.MediumTests;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.CancelableProgressable;
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.ZooDefs.Ids;
48  import org.junit.After;
49  import org.junit.Before;
50  import org.junit.Test;
51  import org.junit.experimental.categories.Category;
52  
53  @Category(MediumTests.class)
54  public class TestSplitLogWorker {
55    private static final Log LOG = LogFactory.getLog(TestSplitLogWorker.class);
56    static {
57      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
58    }
59    private final static HBaseTestingUtility TEST_UTIL =
60      new HBaseTestingUtility();
61    private ZooKeeperWatcher zkw;
62    private SplitLogWorker slw;
63  
64    private void waitForCounter(AtomicLong ctr, long oldval, long newval,
65        long timems) {
66      assertTrue("ctr=" + ctr.get() + ", oldval=" + oldval + ", newval=" + newval,
67        waitForCounterBoolean(ctr, oldval, newval, timems));
68    }
69  
70    private boolean waitForCounterBoolean(AtomicLong ctr, long oldval, long newval,
71        long timems) {
72      long curt = System.currentTimeMillis();
73      long endt = curt + timems;
74      while (curt < endt) {
75        if (ctr.get() == oldval) {
76          try {
77            Thread.sleep(10);
78          } catch (InterruptedException e) {
79          }
80          curt = System.currentTimeMillis();
81        } else {
82          assertEquals(newval, ctr.get());
83          return true;
84        }
85      }
86      return false;
87    }
88  
89    @Before
90    public void setup() throws Exception {
91      TEST_UTIL.startMiniZKCluster();
92      zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
93          "split-log-worker-tests", null);
94      ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
95      ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
96      assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
97      LOG.debug(zkw.baseZNode + " created");
98      ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
99      assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
100     LOG.debug(zkw.splitLogZNode + " created");
101     resetCounters();
102 
103   }
104 
105   @After
106   public void teardown() throws Exception {
107     TEST_UTIL.shutdownMiniZKCluster();
108   }
109 
110   SplitLogWorker.TaskExecutor neverEndingTask =
111     new SplitLogWorker.TaskExecutor() {
112 
113       @Override
114       public Status exec(String name, CancelableProgressable p) {
115         while (true) {
116           try {
117             Thread.sleep(1000);
118           } catch (InterruptedException e) {
119             return Status.PREEMPTED;
120           }
121           if (!p.progress()) {
122             return Status.PREEMPTED;
123           }
124         }
125       }
126 
127   };
128 
129   @Test
130   public void testAcquireTaskAtStartup() throws Exception {
131     LOG.info("testAcquireTaskAtStartup");
132     ZKSplitLog.Counters.resetCounters();
133 
134     zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tatas"),
135         TaskState.TASK_UNASSIGNED.get("mgr"), Ids.OPEN_ACL_UNSAFE,
136         CreateMode.PERSISTENT);
137 
138     SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
139       "rs", neverEndingTask);
140     slw.start();
141     try {
142       waitForCounter(tot_wkr_task_acquired, 0, 1, 1500);
143       assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
144         ZKSplitLog.getEncodedNodeName(zkw, "tatas")), "rs"));
145     } finally {
146       stopSplitLogWorker(slw);
147     }
148   }
149 
150   private void stopSplitLogWorker(final SplitLogWorker slw)
151   throws InterruptedException {
152     if (slw != null) {
153       slw.stop();
154       slw.worker.join(3000);
155       if (slw.worker.isAlive()) {
156         assertTrue(("Could not stop the worker thread slw=" + slw) == null);
157       }
158     }
159   }
160 
161   @Test
162   public void testRaceForTask() throws Exception {
163     LOG.info("testRaceForTask");
164     ZKSplitLog.Counters.resetCounters();
165 
166     zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "trft"),
167         TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
168         CreateMode.PERSISTENT);
169 
170     SplitLogWorker slw1 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
171         "svr1", neverEndingTask);
172     SplitLogWorker slw2 = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
173         "svr2", neverEndingTask);
174     slw1.start();
175     slw2.start();
176     try {
177       waitForCounter(tot_wkr_task_acquired, 0, 1, 1500);
178       // Assert that either the tot_wkr_failed_to_grab_task_owned count was set of if
179       // not it, that we fell through to the next counter in line and it was set.
180       assertTrue(waitForCounterBoolean(tot_wkr_failed_to_grab_task_owned, 0, 1, 1500) ||
181         tot_wkr_failed_to_grab_task_lost_race.get() == 1);
182       assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
183         ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr1") ||
184         TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
185             ZKSplitLog.getEncodedNodeName(zkw, "trft")), "svr2"));
186     } finally {
187       stopSplitLogWorker(slw1);
188       stopSplitLogWorker(slw2);
189     }
190   }
191 
192   @Test
193   public void testPreemptTask() throws Exception {
194     LOG.info("testPreemptTask");
195     ZKSplitLog.Counters.resetCounters();
196 
197     SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
198         "tpt_svr", neverEndingTask);
199     slw.start();
200     try {
201       Thread.yield(); // let the worker start
202       Thread.sleep(100);
203 
204       // this time create a task node after starting the splitLogWorker
205       zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
206         TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
207         CreateMode.PERSISTENT);
208 
209       waitForCounter(tot_wkr_task_acquired, 0, 1, 1500);
210       assertEquals(1, slw.taskReadySeq);
211       assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
212         ZKSplitLog.getEncodedNodeName(zkw, "tpt_task")), "tpt_svr"));
213 
214       ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tpt_task"),
215         TaskState.TASK_UNASSIGNED.get("manager"));
216       waitForCounter(tot_wkr_preempt_task, 0, 1, 1500);
217     } finally {
218       stopSplitLogWorker(slw);
219     }
220   }
221 
222   @Test
223   public void testMultipleTasks() throws Exception {
224     LOG.info("testMultipleTasks");
225     ZKSplitLog.Counters.resetCounters();
226     SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
227         "tmt_svr", neverEndingTask);
228     slw.start();
229     try {
230       Thread.yield(); // let the worker start
231       Thread.sleep(100);
232 
233       zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
234         TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
235         CreateMode.PERSISTENT);
236 
237       waitForCounter(tot_wkr_task_acquired, 0, 1, 1500);
238       // now the worker is busy doing the above task
239 
240       // create another task
241       zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2"),
242         TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
243         CreateMode.PERSISTENT);
244 
245       // preempt the first task, have it owned by another worker
246       ZKUtil.setData(zkw, ZKSplitLog.getEncodedNodeName(zkw, "tmt_task"),
247         TaskState.TASK_OWNED.get("another-worker"));
248       waitForCounter(tot_wkr_preempt_task, 0, 1, 1500);
249 
250       waitForCounter(tot_wkr_task_acquired, 1, 2, 1500);
251       assertEquals(2, slw.taskReadySeq);
252       assertTrue(TaskState.TASK_OWNED.equals(ZKUtil.getData(zkw,
253         ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2")), "tmt_svr"));
254     } finally {
255       stopSplitLogWorker(slw);
256     }
257   }
258 
259   @Test
260   public void testRescan() throws Exception {
261     LOG.info("testRescan");
262     ZKSplitLog.Counters.resetCounters();
263     slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(),
264         "svr", neverEndingTask);
265     slw.start();
266     Thread.yield(); // let the worker start
267     Thread.sleep(200);
268 
269     String task = ZKSplitLog.getEncodedNodeName(zkw, "task");
270     zkw.getRecoverableZooKeeper().create(task,
271       TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
272       CreateMode.PERSISTENT);
273 
274     waitForCounter(tot_wkr_task_acquired, 0, 1, 1500);
275     // now the worker is busy doing the above task
276 
277     // preempt the task, have it owned by another worker
278     ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
279     waitForCounter(tot_wkr_preempt_task, 0, 1, 1500);
280 
281     // create a RESCAN node
282     String rescan = ZKSplitLog.getEncodedNodeName(zkw, "RESCAN");
283     rescan = zkw.getRecoverableZooKeeper().create(rescan,
284       TaskState.TASK_UNASSIGNED.get("manager"), Ids.OPEN_ACL_UNSAFE,
285       CreateMode.PERSISTENT_SEQUENTIAL);
286 
287     waitForCounter(tot_wkr_task_acquired, 1, 2, 1500);
288     // RESCAN node might not have been processed if the worker became busy
289     // with the above task. preempt the task again so that now the RESCAN
290     // node is processed
291     ZKUtil.setData(zkw, task, TaskState.TASK_UNASSIGNED.get("manager"));
292     waitForCounter(tot_wkr_preempt_task, 1, 2, 1500);
293     waitForCounter(tot_wkr_task_acquired_rescan, 0, 1, 1500);
294 
295     List<String> nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode);
296     LOG.debug(nodes);
297     int num = 0;
298     for (String node : nodes) {
299       num++;
300       if (node.startsWith("RESCAN")) {
301         String name = ZKSplitLog.getEncodedNodeName(zkw, node);
302         String fn = ZKSplitLog.getFileName(name);
303         byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.splitLogZNode, fn));
304         String datastr = Bytes.toString(data);
305         assertTrue("data=" + datastr, TaskState.TASK_DONE.equals(data, "svr"));
306       }
307     }
308     assertEquals(2, num);
309   }
310 
311   @org.junit.Rule
312   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
313     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
314 }
315