View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.procedure2;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.util.concurrent.atomic.AtomicBoolean;
25  import java.util.concurrent.CountDownLatch;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.FileStatus;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
33  import org.apache.hadoop.hbase.ProcedureInfo;
34  import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
35  import org.apache.hadoop.hbase.testclassification.SmallTests;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
38  import org.apache.hadoop.hbase.util.Threads;
39  import org.junit.After;
40  import org.junit.Before;
41  import org.junit.Test;
42  import org.junit.experimental.categories.Category;
43  
44  import static org.junit.Assert.assertEquals;
45  import static org.junit.Assert.assertFalse;
46  import static org.junit.Assert.assertTrue;
47  
48  @Category(SmallTests.class)
49  public class TestProcedureRecovery {
50    private static final Log LOG = LogFactory.getLog(TestProcedureRecovery.class);
51  
52    private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
53  
54    private static TestProcEnv procEnv;
55    private static ProcedureExecutor<TestProcEnv> procExecutor;
56    private static ProcedureStore procStore;
57    private static int procSleepInterval;
58  
59    private HBaseCommonTestingUtility htu;
60    private FileSystem fs;
61    private Path testDir;
62    private Path logDir;
63  
64    @Before
65    public void setUp() throws IOException {
66      htu = new HBaseCommonTestingUtility();
67      testDir = htu.getDataTestDir();
68      fs = testDir.getFileSystem(htu.getConfiguration());
69      assertTrue(testDir.depth() > 1);
70  
71      logDir = new Path(testDir, "proc-logs");
72      procEnv = new TestProcEnv();
73      procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), logDir);
74      procExecutor = new ProcedureExecutor(htu.getConfiguration(), procEnv, procStore);
75      procExecutor.testing = new ProcedureExecutor.Testing();
76      procStore.start(PROCEDURE_EXECUTOR_SLOTS);
77      procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
78      procSleepInterval = 0;
79    }
80  
81    @After
82    public void tearDown() throws IOException {
83      procExecutor.stop();
84      procStore.stop(false);
85      fs.delete(logDir, true);
86    }
87  
88    private void restart() throws Exception {
89      dumpLogDirState();
90      ProcedureTestingUtility.restart(procExecutor);
91      dumpLogDirState();
92    }
93  
94    public static class TestSingleStepProcedure extends SequentialProcedure<TestProcEnv> {
95      private int step = 0;
96  
97      public TestSingleStepProcedure() { }
98  
99      @Override
100     protected Procedure[] execute(TestProcEnv env) throws InterruptedException {
101       env.waitOnLatch();
102       LOG.debug("execute procedure " + this + " step=" + step);
103       step++;
104       setResult(Bytes.toBytes(step));
105       return null;
106     }
107 
108     @Override
109     protected void rollback(TestProcEnv env) { }
110 
111     @Override
112     protected boolean abort(TestProcEnv env) { return true; }
113   }
114 
115   public static class BaseTestStepProcedure extends SequentialProcedure<TestProcEnv> {
116     private AtomicBoolean abort = new AtomicBoolean(false);
117     private int step = 0;
118 
119     @Override
120     protected Procedure[] execute(TestProcEnv env) throws InterruptedException {
121       env.waitOnLatch();
122       LOG.debug("execute procedure " + this + " step=" + step);
123       ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
124       step++;
125       Threads.sleepWithoutInterrupt(procSleepInterval);
126       if (isAborted()) {
127         setFailure(new RemoteProcedureException(getClass().getName(),
128           new ProcedureAbortedException(
129             "got an abort at " + getClass().getName() + " step=" + step)));
130         return null;
131       }
132       return null;
133     }
134 
135     @Override
136     protected void rollback(TestProcEnv env) {
137       LOG.debug("rollback procedure " + this + " step=" + step);
138       ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
139       step++;
140     }
141 
142     @Override
143     protected boolean abort(TestProcEnv env) {
144       abort.set(true);
145       return true;
146     }
147 
148     private boolean isAborted() {
149       boolean aborted = abort.get();
150       BaseTestStepProcedure proc = this;
151       while (proc.hasParent() && !aborted) {
152         proc = (BaseTestStepProcedure)procExecutor.getProcedure(proc.getParentProcId());
153         aborted = proc.isAborted();
154       }
155       return aborted;
156     }
157   }
158 
159   public static class TestMultiStepProcedure extends BaseTestStepProcedure {
160     public TestMultiStepProcedure() { }
161 
162     @Override
163     public Procedure[] execute(TestProcEnv env) throws InterruptedException {
164       super.execute(env);
165       return isFailed() ? null : new Procedure[] { new Step1Procedure() };
166     }
167 
168     public static class Step1Procedure extends BaseTestStepProcedure {
169       public Step1Procedure() { }
170 
171       @Override
172       protected Procedure[] execute(TestProcEnv env) throws InterruptedException {
173         super.execute(env);
174         return isFailed() ? null : new Procedure[] { new Step2Procedure() };
175       }
176     }
177 
178     public static class Step2Procedure extends BaseTestStepProcedure {
179       public Step2Procedure() { }
180     }
181   }
182 
183   @Test
184   public void testNoopLoad() throws Exception {
185     restart();
186   }
187 
188   @Test(timeout=30000)
189   public void testSingleStepProcRecovery() throws Exception {
190     Procedure proc = new TestSingleStepProcedure();
191     procExecutor.testing.killBeforeStoreUpdate = true;
192     long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
193     assertFalse(procExecutor.isRunning());
194     procExecutor.testing.killBeforeStoreUpdate = false;
195 
196     // Restart and verify that the procedures restart
197     long restartTs = EnvironmentEdgeManager.currentTime();
198     restart();
199     waitProcedure(procId);
200     ProcedureInfo result = procExecutor.getResult(procId);
201     assertTrue(result.getLastUpdate() > restartTs);
202     ProcedureTestingUtility.assertProcNotFailed(result);
203     assertEquals(1, Bytes.toInt(result.getResult()));
204     long resultTs = result.getLastUpdate();
205 
206     // Verify that after another restart the result is still there
207     restart();
208     result = procExecutor.getResult(procId);
209     ProcedureTestingUtility.assertProcNotFailed(result);
210     assertEquals(resultTs, result.getLastUpdate());
211     assertEquals(1, Bytes.toInt(result.getResult()));
212   }
213 
214   @Test(timeout=30000)
215   public void testMultiStepProcRecovery() throws Exception {
216     // Step 0 - kill
217     Procedure proc = new TestMultiStepProcedure();
218     long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
219     assertFalse(procExecutor.isRunning());
220 
221     // Step 0 exec && Step 1 - kill
222     restart();
223     waitProcedure(procId);
224     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
225     assertFalse(procExecutor.isRunning());
226 
227     // Step 1 exec && step 2 - kill
228     restart();
229     waitProcedure(procId);
230     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
231     assertFalse(procExecutor.isRunning());
232 
233     // Step 2 exec
234     restart();
235     waitProcedure(procId);
236     assertTrue(procExecutor.isRunning());
237 
238     // The procedure is completed
239     ProcedureInfo result = procExecutor.getResult(procId);
240     ProcedureTestingUtility.assertProcNotFailed(result);
241   }
242 
243   @Test(timeout=30000)
244   public void testMultiStepRollbackRecovery() throws Exception {
245     // Step 0 - kill
246     Procedure proc = new TestMultiStepProcedure();
247     long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
248     assertFalse(procExecutor.isRunning());
249 
250     // Step 0 exec && Step 1 - kill
251     restart();
252     waitProcedure(procId);
253     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
254     assertFalse(procExecutor.isRunning());
255 
256     // Step 1 exec && step 2 - kill
257     restart();
258     waitProcedure(procId);
259     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
260     assertFalse(procExecutor.isRunning());
261 
262     // Step 2 exec - rollback - kill
263     procSleepInterval = 2500;
264     restart();
265     assertTrue(procExecutor.abort(procId));
266     waitProcedure(procId);
267     assertFalse(procExecutor.isRunning());
268 
269     // rollback - kill
270     restart();
271     waitProcedure(procId);
272     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
273     assertFalse(procExecutor.isRunning());
274 
275     // rollback - complete
276     restart();
277     waitProcedure(procId);
278     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
279     assertFalse(procExecutor.isRunning());
280 
281     // Restart the executor and get the result
282     restart();
283     waitProcedure(procId);
284 
285     // The procedure is completed
286     ProcedureInfo result = procExecutor.getResult(procId);
287     ProcedureTestingUtility.assertIsAbortException(result);
288   }
289 
290   public static class TestStateMachineProcedure
291       extends StateMachineProcedure<TestProcEnv, TestStateMachineProcedure.State> {
292     enum State { STATE_1, STATE_2, STATE_3, DONE }
293 
294     public TestStateMachineProcedure() {}
295 
296     public TestStateMachineProcedure(final boolean testSubmitChildProc) {
297       this.submitChildProc = testSubmitChildProc;
298     }
299 
300     private AtomicBoolean aborted = new AtomicBoolean(false);
301     private int iResult = 0;
302     private boolean submitChildProc = false;
303 
304     @Override
305     protected StateMachineProcedure.Flow executeFromState(TestProcEnv env, State state) {
306       switch (state) {
307         case STATE_1:
308           LOG.info("execute step 1 " + this);
309           setNextState(State.STATE_2);
310           iResult += 3;
311           break;
312         case STATE_2:
313           LOG.info("execute step 2 " + this);
314           if (submitChildProc) {
315             addChildProcedure(new TestStateMachineProcedure(), new TestStateMachineProcedure());
316             setNextState(State.DONE);
317           } else {
318             setNextState(State.STATE_3);
319           }
320           iResult += 5;
321           break;
322         case STATE_3:
323           LOG.info("execute step 3 " + this);
324           Threads.sleepWithoutInterrupt(procSleepInterval);
325           if (aborted.get()) {
326             LOG.info("aborted step 3 " + this);
327             setAbortFailure("test", "aborted");
328             break;
329           }
330           setNextState(State.DONE);
331           iResult += 7;
332           break;
333         case DONE:
334           if (submitChildProc) {
335             addChildProcedure(new TestStateMachineProcedure());
336           }
337           iResult += 11;
338           setResult(Bytes.toBytes(iResult));
339           return Flow.NO_MORE_STATE;
340         default:
341           throw new UnsupportedOperationException();
342       }
343       return Flow.HAS_MORE_STATE;
344     }
345 
346     @Override
347     protected void rollbackState(TestProcEnv env, final State state) {
348       switch (state) {
349         case STATE_1:
350           LOG.info("rollback step 1 " + this);
351           break;
352         case STATE_2:
353           LOG.info("rollback step 2 " + this);
354           break;
355         case STATE_3:
356           LOG.info("rollback step 3 " + this);
357           break;
358         default:
359           throw new UnsupportedOperationException();
360       }
361     }
362 
363     @Override
364     protected State getState(final int stateId) {
365       return State.values()[stateId];
366     }
367 
368     @Override
369     protected int getStateId(final State state) {
370       return state.ordinal();
371     }
372 
373     @Override
374     protected State getInitialState() {
375       return State.STATE_1;
376     }
377 
378     @Override
379     protected boolean abort(TestProcEnv env) {
380       aborted.set(true);
381       return true;
382     }
383 
384     @Override
385     protected void serializeStateData(final OutputStream stream) throws IOException {
386       super.serializeStateData(stream);
387       stream.write(Bytes.toBytes(iResult));
388     }
389 
390     @Override
391     protected void deserializeStateData(final InputStream stream) throws IOException {
392       super.deserializeStateData(stream);
393       byte[] data = new byte[4];
394       stream.read(data);
395       iResult = Bytes.toInt(data);
396     }
397   }
398 
399   @Test(timeout=30000)
400   public void testStateMachineMultipleLevel() throws Exception {
401     long procId = procExecutor.submitProcedure(new TestStateMachineProcedure(true));
402     // Wait the completion
403     ProcedureTestingUtility.waitProcedure(procExecutor, procId);
404     ProcedureInfo result = procExecutor.getResult(procId);
405     ProcedureTestingUtility.assertProcNotFailed(result);
406     assertEquals(19, Bytes.toInt(result.getResult()));
407     assertEquals(4, procExecutor.getLastProcId());
408   }
409 
410   @Test(timeout=30000)
411   public void testStateMachineRecovery() throws Exception {
412     ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
413     ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
414 
415     // Step 1 - kill
416     Procedure proc = new TestStateMachineProcedure();
417     long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
418     assertFalse(procExecutor.isRunning());
419 
420     // Step 1 exec && Step 2 - kill
421     restart();
422     waitProcedure(procId);
423     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
424     assertFalse(procExecutor.isRunning());
425 
426     // Step 2 exec && step 3 - kill
427     restart();
428     waitProcedure(procId);
429     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
430     assertFalse(procExecutor.isRunning());
431 
432     // Step 3 exec
433     restart();
434     waitProcedure(procId);
435     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
436     assertFalse(procExecutor.isRunning());
437 
438     restart();
439     waitProcedure(procId);
440     assertTrue(procExecutor.isRunning());
441 
442     // The procedure is completed
443     ProcedureInfo result = procExecutor.getResult(procId);
444     ProcedureTestingUtility.assertProcNotFailed(result);
445     assertEquals(26, Bytes.toInt(result.getResult()));
446   }
447 
448   @Test(timeout=30000)
449   public void testStateMachineRollbackRecovery() throws Exception {
450     ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
451     ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
452 
453     // Step 1 - kill
454     Procedure proc = new TestStateMachineProcedure();
455     long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
456     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
457     assertFalse(procExecutor.isRunning());
458 
459     // Step 1 exec && Step 2 - kill
460     restart();
461     waitProcedure(procId);
462     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
463     assertFalse(procExecutor.isRunning());
464 
465     // Step 2 exec && step 3 - kill
466     restart();
467     waitProcedure(procId);
468     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
469     assertFalse(procExecutor.isRunning());
470 
471     // Step 3 exec - rollback step 3 - kill
472     procSleepInterval = 2500;
473     restart();
474     assertTrue(procExecutor.abort(procId));
475     waitProcedure(procId);
476     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
477     assertFalse(procExecutor.isRunning());
478 
479     // Rollback step 3 - rollback step 2 - kill
480     restart();
481     waitProcedure(procId);
482     assertFalse(procExecutor.isRunning());
483     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
484 
485     // Rollback step 2 - step 1 - kill
486     restart();
487     waitProcedure(procId);
488     assertFalse(procExecutor.isRunning());
489     ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
490 
491     // Rollback step 1 - complete
492     restart();
493     waitProcedure(procId);
494     assertTrue(procExecutor.isRunning());
495 
496     // The procedure is completed
497     ProcedureInfo result = procExecutor.getResult(procId);
498     ProcedureTestingUtility.assertIsAbortException(result);
499   }
500 
501   private void waitProcedure(final long procId) {
502     ProcedureTestingUtility.waitProcedure(procExecutor, procId);
503     dumpLogDirState();
504   }
505 
506   private void dumpLogDirState() {
507     try {
508       FileStatus[] files = fs.listStatus(logDir);
509       if (files != null && files.length > 0) {
510         for (FileStatus file: files) {
511           assertTrue(file.toString(), file.isFile());
512           LOG.debug("log file " + file.getPath() + " size=" + file.getLen());
513         }
514       } else {
515         LOG.debug("no files under: " + logDir);
516       }
517     } catch (IOException e) {
518       LOG.warn("Unable to dump " + logDir, e);
519     }
520   }
521 
522   private static class TestProcEnv {
523     private CountDownLatch latch = null;
524 
525     /**
526      * set/unset a latch. every procedure execute() step will wait on the latch if any.
527      */
528     public void setWaitLatch(CountDownLatch latch) {
529       this.latch = latch;
530     }
531 
532     public void waitOnLatch() throws InterruptedException {
533       if (latch != null) {
534         latch.await();
535       }
536     }
537   }
538 }