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