You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2016/10/19 14:17:24 UTC

hbase git commit: HBASE-16864 Procedure v2 - Fix StateMachineProcedure support for child procs at last step

Repository: hbase
Updated Branches:
  refs/heads/master 67f1ac1f8 -> 82a238418


HBASE-16864 Procedure v2 - Fix StateMachineProcedure support for child procs at last step


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/82a23841
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/82a23841
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/82a23841

Branch: refs/heads/master
Commit: 82a2384187180990e1863746bf1793b4409cb305
Parents: 67f1ac1
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Oct 19 06:58:41 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Oct 19 06:58:41 2016 -0700

----------------------------------------------------------------------
 .../hbase/procedure2/StateMachineProcedure.java |  20 +-
 .../procedure2/ProcedureTestingUtility.java     |  73 ++++++-
 .../procedure2/TestStateMachineProcedure.java   | 195 +++++++++++++++++++
 3 files changed, 277 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82a23841/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 2ec4418..3f9a7b7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -48,6 +48,8 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
     extends Procedure<TEnvironment> {
   private static final Log LOG = LogFactory.getLog(StateMachineProcedure.class);
 
+  private static final int EOF_STATE = Integer.MIN_VALUE;
+
   private final AtomicBoolean aborted = new AtomicBoolean(false);
 
   private Flow stateFlow = Flow.HAS_MORE_STATE;
@@ -150,6 +152,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
       }
 
       stateFlow = executeFromState(env, state);
+      if (!hasMoreState()) setNextState(EOF_STATE);
 
       if (subProcList != null && subProcList.size() != 0) {
         Procedure[] subProcedures = subProcList.toArray(new Procedure[subProcList.size()]);
@@ -166,6 +169,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
   @Override
   protected void rollback(final TEnvironment env)
       throws IOException, InterruptedException {
+    if (isEofState()) stateCount--;
     try {
       updateTimestamp();
       rollbackState(env, getCurrentState());
@@ -175,13 +179,22 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
     }
   }
 
+  private boolean isEofState() {
+    return stateCount > 0 && states[stateCount-1] == EOF_STATE;
+  }
+
   @Override
   protected boolean abort(final TEnvironment env) {
+    final boolean isDebugEnabled = LOG.isDebugEnabled();
     final TState state = getCurrentState();
     if (isRollbackSupported(state)) {
-      LOG.debug("abort requested for " + getClass().getSimpleName() + " state=" + state);
+      if (isDebugEnabled) {
+        LOG.debug("abort requested for " + this + " state=" + state);
+      }
       aborted.set(true);
       return true;
+    } else if (isDebugEnabled) {
+      LOG.debug("ignoring abort request on state=" + state + " for " + this);
     }
     return false;
   }
@@ -226,7 +239,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
   @Override
   protected void toStringState(StringBuilder builder) {
     super.toStringState(builder);
-    if (!isFinished() && getCurrentState() != null) {
+    if (!isFinished() && !isEofState() && getCurrentState() != null) {
       builder.append(":").append(getCurrentState());
     }
   }
@@ -249,6 +262,9 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
       for (int i = 0; i < stateCount; ++i) {
         states[i] = data.getState(i);
       }
+      if (isEofState()) {
+        stateFlow = Flow.NO_MORE_STATE;
+      }
     } else {
       states = null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a23841/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 0b4e4ed..2d27c59 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -208,24 +208,31 @@ public class ProcedureTestingUtility {
     assertFalse("found exception: " + result.getException(), result.isFailed());
   }
 
-  public static void assertIsAbortException(final ProcedureInfo result) {
+  public static <TEnv> Throwable assertProcFailed(final ProcedureExecutor<TEnv> procExecutor,
+      final long procId) {
+    ProcedureInfo result = procExecutor.getResult(procId);
+    assertTrue("expected procedure result", result != null);
+    return assertProcFailed(result);
+  }
+
+  public static Throwable assertProcFailed(final ProcedureInfo result) {
     assertEquals(true, result.isFailed());
-    LOG.info(result.getException().getMessage());
-    Throwable cause = result.getException().getCause();
+    LOG.info("procId=" + result.getProcId() + " exception: " + result.getException().getMessage());
+    return getExceptionCause(result);
+  }
+
+  public static void assertIsAbortException(final ProcedureInfo result) {
+    Throwable cause = assertProcFailed(result);
     assertTrue("expected abort exception, got "+ cause, cause instanceof ProcedureAbortedException);
   }
 
   public static void assertIsTimeoutException(final ProcedureInfo result) {
-    assertEquals(true, result.isFailed());
-    LOG.info(result.getException().getMessage());
-    Throwable cause = result.getException();
+    Throwable cause = assertProcFailed(result);
     assertTrue("expected TimeoutIOException, got " + cause, cause instanceof TimeoutIOException);
   }
 
   public static void assertIsIllegalArgumentException(final ProcedureInfo result) {
-    assertEquals(true, result.isFailed());
-    LOG.info(result.getException().getMessage());
-    Throwable cause = getExceptionCause(result);
+    Throwable cause = assertProcFailed(result);
     assertTrue("expected IllegalArgumentIOException, got " + cause,
       cause instanceof IllegalArgumentIOException);
   }
@@ -236,6 +243,54 @@ public class ProcedureTestingUtility {
     return cause == null ? procInfo.getException() : cause;
   }
 
+  /**
+   * Run through all procedure flow states TWICE while also restarting
+   * procedure executor at each step; i.e force a reread of procedure store.
+   *
+   *<p>It does
+   * <ol><li>Execute step N - kill the executor before store update
+   * <li>Restart executor/store
+   * <li>Execute step N - and then save to store
+   * </ol>
+   *
+   *<p>This is a good test for finding state that needs persisting and steps that are not
+   * idempotent.
+   */
+  public static <TEnv> void testRecoveryAndDoubleExecution(final ProcedureExecutor<TEnv> procExec,
+      final long procId) throws Exception {
+    testRecoveryAndDoubleExecution(procExec, procId, false);
+  }
+
+  public static <TEnv> void testRecoveryAndDoubleExecution(final ProcedureExecutor<TEnv> procExec,
+      final long procId, final boolean expectFailure) throws Exception {
+    testRecoveryAndDoubleExecution(procExec, procId, expectFailure, null);
+  }
+
+  public static <TEnv> void testRecoveryAndDoubleExecution(final ProcedureExecutor<TEnv> procExec,
+      final long procId, final boolean expectFailure, final Runnable customRestart)
+      throws Exception {
+    final Procedure proc = procExec.getProcedure(procId);
+    waitProcedure(procExec, procId);
+    assertEquals(false, procExec.isRunning());
+
+    for (int i = 0; !procExec.isFinished(procId); ++i) {
+      LOG.info("Restart " + i + " exec state: " + proc);
+      if (customRestart != null) {
+        customRestart.run();
+      } else {
+        restart(procExec);
+      }
+      waitProcedure(procExec, procId);
+    }
+
+    assertEquals(true, procExec.isRunning());
+    if (expectFailure) {
+      assertProcFailed(procExec, procId);
+    } else {
+      assertProcNotFailed(procExec, procId);
+    }
+  }
+
   public static class NoopProcedure<TEnv> extends Procedure<TEnv> {
     public NoopProcedure() {}
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a23841/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
new file mode 100644
index 0000000..4b36c76
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestStateMachineProcedure.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.procedure2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.NoopProcedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestStateMachineProcedure {
+  private static final Log LOG = LogFactory.getLog(TestStateMachineProcedure.class);
+
+  private static final Exception TEST_FAILURE_EXCEPTION = new Exception("test failure") {
+    @Override
+    public boolean equals(final Object other) {
+      if (this == other) return true;
+      if (!(other instanceof Exception)) return false;
+      // we are going to serialize the exception in the test,
+      // so the instance comparison will not match
+      return getMessage().equals(((Exception)other).getMessage());
+    }
+
+    @Override
+    public int hashCode() {
+      return getMessage().hashCode();
+    }
+  };
+
+  private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
+
+  private ProcedureExecutor<TestProcEnv> procExecutor;
+  private ProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+
+    logDir = new Path(testDir, "proc-logs");
+    procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor(htu.getConfiguration(), new TestProcEnv(), procStore);
+    procStore.start(PROCEDURE_EXECUTOR_SLOTS);
+    procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExecutor, false);
+    assertTrue("expected executor to be running", procExecutor.isRunning());
+
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  @Test
+  public void testChildOnLastStep() {
+    long procId = procExecutor.submitProcedure(new TestSMProcedure());
+    ProcedureTestingUtility.waitProcedure(procExecutor, procId);
+    assertEquals(3, procExecutor.getEnvironment().execCount.get());
+    assertEquals(0, procExecutor.getEnvironment().rollbackCount.get());
+    ProcedureTestingUtility.assertProcNotFailed(procExecutor, procId);
+  }
+
+  @Test
+  public void testChildOnLastStepDoubleExecution() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExecutor, true);
+    long procId = procExecutor.submitProcedure(new TestSMProcedure());
+    ProcedureTestingUtility.testRecoveryAndDoubleExecution(procExecutor, procId);
+    assertEquals(6, procExecutor.getEnvironment().execCount.get());
+    assertEquals(0, procExecutor.getEnvironment().rollbackCount.get());
+    ProcedureTestingUtility.assertProcNotFailed(procExecutor, procId);
+  }
+
+  @Test
+  public void testChildOnLastStepWithRollback() {
+    procExecutor.getEnvironment().triggerChildRollback = true;
+    long procId = procExecutor.submitProcedure(new TestSMProcedure());
+    ProcedureTestingUtility.waitProcedure(procExecutor, procId);
+    assertEquals(3, procExecutor.getEnvironment().execCount.get());
+    assertEquals(3, procExecutor.getEnvironment().rollbackCount.get());
+    Throwable cause = ProcedureTestingUtility.assertProcFailed(procExecutor, procId);
+    assertEquals(TEST_FAILURE_EXCEPTION, cause);
+  }
+
+  @Test
+  public void testChildOnLastStepWithRollbackDoubleExecution() throws Exception {
+    procExecutor.getEnvironment().triggerChildRollback = true;
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExecutor, true);
+    long procId = procExecutor.submitProcedure(new TestSMProcedure());
+    ProcedureTestingUtility.testRecoveryAndDoubleExecution(procExecutor, procId, true);
+    assertEquals(6, procExecutor.getEnvironment().execCount.get());
+    assertEquals(6, procExecutor.getEnvironment().rollbackCount.get());
+    Throwable cause = ProcedureTestingUtility.assertProcFailed(procExecutor, procId);
+    assertEquals(TEST_FAILURE_EXCEPTION, cause);
+  }
+
+  public enum TestSMProcedureState { STEP_1, STEP_2 };
+  public static class TestSMProcedure
+      extends StateMachineProcedure<TestProcEnv, TestSMProcedureState> {
+    protected Flow executeFromState(TestProcEnv env, TestSMProcedureState state) {
+      LOG.info("EXEC " + state + " " + this);
+      env.execCount.incrementAndGet();
+      switch (state) {
+        case STEP_1:
+          setNextState(TestSMProcedureState.STEP_2);
+          break;
+        case STEP_2:
+          addChildProcedure(new SimpleChildProcedure());
+          return Flow.NO_MORE_STATE;
+      }
+      return Flow.HAS_MORE_STATE;
+    }
+
+    protected void rollbackState(TestProcEnv env, TestSMProcedureState state) {
+      LOG.info("ROLLBACK " + state + " " + this);
+      env.rollbackCount.incrementAndGet();
+    }
+
+    protected TestSMProcedureState getState(int stateId) {
+      return TestSMProcedureState.values()[stateId];
+    }
+
+    protected int getStateId(TestSMProcedureState state) {
+      return state.ordinal();
+    }
+
+    protected TestSMProcedureState getInitialState() {
+      return TestSMProcedureState.STEP_1;
+    }
+  }
+
+  public static class SimpleChildProcedure extends NoopProcedure<TestProcEnv> {
+    protected Procedure[] execute(TestProcEnv env) {
+      LOG.info("EXEC " + this);
+      env.execCount.incrementAndGet();
+      if (env.triggerChildRollback) {
+        setFailure("test-failure", TEST_FAILURE_EXCEPTION);
+      }
+      return null;
+    }
+
+    @Override
+    protected void rollback(TestProcEnv env) {
+      LOG.info("ROLLBACK " + this);
+      env.rollbackCount.incrementAndGet();
+    }
+  }
+
+  public class TestProcEnv {
+    AtomicInteger execCount = new AtomicInteger(0);
+    AtomicInteger rollbackCount = new AtomicInteger(0);
+    boolean triggerChildRollback = false;
+  }
+}