You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xy...@apache.org on 2017/08/18 01:47:12 UTC

hadoop git commit: HDFS-12313. Ozone: SCM: move container/pipeline StateMachine to the right package. Contributed by Xiaoyu Yao.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 293c425b2 -> 6c1e9ab2a


HDFS-12313. Ozone: SCM: move container/pipeline StateMachine to the right package. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6c1e9ab2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6c1e9ab2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6c1e9ab2

Branch: refs/heads/HDFS-7240
Commit: 6c1e9ab2a434047b70c30f2bedd65b8e4b25c9d8
Parents: 293c425
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Wed Aug 16 22:07:16 2017 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Aug 17 11:27:54 2017 -0700

----------------------------------------------------------------------
 .../InvalidStateTransitionException.java        |  42 --------
 .../helpers/StateMachine/StateMachine.java      |  69 ------------
 .../org/apache/hadoop/scm/TestStateMachine.java |  95 -----------------
 .../hadoop/ozone/common/package-info.java       |  21 ++++
 .../InvalidStateTransitionException.java        |  42 ++++++++
 .../ozone/common/statemachine/StateMachine.java |  68 ++++++++++++
 .../ozone/common/statemachine/package-info.java |  21 ++++
 .../hadoop/ozone/common/TestStateMachine.java   | 104 +++++++++++++++++++
 8 files changed, 256 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/InvalidStateTransitionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/InvalidStateTransitionException.java b/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/InvalidStateTransitionException.java
deleted file mode 100644
index 1fab16b..0000000
--- a/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/InvalidStateTransitionException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.scm.container.common.helpers.StateMachine;
-
-/**
- * Class wraps invalid state transition exception.
- */
-public class InvalidStateTransitionException extends Exception {
-  private Enum<?> currentState;
-  private Enum<?> event;
-
-  public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState + " state.");
-    this.currentState = currentState;
-    this.event = event;
-  }
-
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-
-  public Enum<?> getEvent() {
-    return event;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/StateMachine.java b/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/StateMachine.java
deleted file mode 100644
index 1d5436f..0000000
--- a/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/container/common/helpers/StateMachine/StateMachine.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.scm.container.common.helpers.StateMachine;
-
-import com.google.common.base.Supplier;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Template class that wraps simple event driven state machine.
- * @param <STATE> states allowed
- * @param <EVENT> events allowed
- */
-public class StateMachine<STATE extends Enum<?>, EVENT extends Enum<?>> {
-  private STATE initialState;
-  private Set<STATE> finalStates;
-
-  private final Cache<EVENT, Map<STATE, STATE>> transitions =
-      CacheBuilder.newBuilder().build(
-          CacheLoader.from((Supplier<Map<STATE, STATE>>) () -> new HashMap()));
-
-  public StateMachine(STATE initState, Set<STATE> finalStates) {
-    this.initialState = initState;
-    this.finalStates = finalStates;
-  }
-
-  public STATE getInitialState() {
-    return initialState;
-  }
-
-  public Set<STATE> getFinalStates() {
-    return finalStates;
-  }
-
-  public STATE getNextState(STATE from, EVENT e)
-      throws InvalidStateTransitionException {
-    STATE target = transitions.getUnchecked(e).get(from);
-    if (target == null) {
-      throw new InvalidStateTransitionException(from, e);
-    }
-    return target;
-  }
-
-  public void addTransition(STATE from, STATE to, EVENT e)
-      throws InvalidStateTransitionException {
-    transitions.getUnchecked(e).put(from, to);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestStateMachine.java b/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestStateMachine.java
deleted file mode 100644
index f9e1d5a..0000000
--- a/hadoop-hdfs-client/src/test/java/org/apache/hadoop/scm/TestStateMachine.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.scm;
-
-import org.apache.commons.collections.SetUtils;
-import org.apache.hadoop.scm.container.common.helpers.StateMachine.InvalidStateTransitionException;
-import org.apache.hadoop.scm.container.common.helpers.StateMachine.StateMachine;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.hadoop.scm.TestStateMachine.STATES.INIT;
-import static org.apache.hadoop.scm.TestStateMachine.STATES.CREATING;
-import static org.apache.hadoop.scm.TestStateMachine.STATES.OPERATIONAL;
-import static org.apache.hadoop.scm.TestStateMachine.STATES.CLOSED;
-import static org.apache.hadoop.scm.TestStateMachine.STATES.CLEANUP;
-import static org.apache.hadoop.scm.TestStateMachine.STATES.FINAL;
-
-public class TestStateMachine {
-
-  public enum STATES {INIT, CREATING, OPERATIONAL, CLOSED, CLEANUP, FINAL};
-
-  public enum EVENTS {ALLOCATE, CREATE, UPDATE, CLOSE, DELETE, TIMEOUT};
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  @Test
-  public void testStateMachineStates() throws InvalidStateTransitionException {
-    Set<STATES> finals = new HashSet<>();
-    finals.add(FINAL);
-
-    StateMachine<STATES, EVENTS> stateMachine =
-        new StateMachine<>(INIT, finals);
-
-    stateMachine.addTransition(INIT, CREATING, EVENTS.ALLOCATE);
-    stateMachine.addTransition(CREATING, OPERATIONAL, EVENTS.CREATE);
-    stateMachine.addTransition(OPERATIONAL, OPERATIONAL, EVENTS.UPDATE);
-    stateMachine.addTransition(OPERATIONAL, CLEANUP, EVENTS.DELETE);
-    stateMachine.addTransition(OPERATIONAL, CLOSED, EVENTS.CLOSE);
-    stateMachine.addTransition(CREATING, CLEANUP, EVENTS.TIMEOUT);
-
-    // Initial and Final states
-    Assert.assertEquals("Initial State", INIT, stateMachine.getInitialState());
-    Assert.assertTrue("Final States", SetUtils.isEqualSet(finals,
-        stateMachine.getFinalStates()));
-
-    // Valid state transitions
-    Assert.assertEquals("STATE should be OPERATIONAL after being created",
-        OPERATIONAL, stateMachine.getNextState(CREATING, EVENTS.CREATE));
-    Assert.assertEquals("STATE should be OPERATIONAL after being updated",
-        OPERATIONAL, stateMachine.getNextState(OPERATIONAL, EVENTS.UPDATE));
-    Assert.assertEquals("STATE should be CLEANUP after being deleted",
-        CLEANUP, stateMachine.getNextState(OPERATIONAL, EVENTS.DELETE));
-    Assert.assertEquals("STATE should be CLEANUP after being timeout",
-        CLEANUP, stateMachine.getNextState(CREATING, EVENTS.TIMEOUT));
-    Assert.assertEquals("STATE should be CLOSED after being closed",
-        CLOSED, stateMachine.getNextState(OPERATIONAL, EVENTS.CLOSE));
-    
-    // Negative cases: invalid transition
-    expectException();
-    stateMachine.getNextState(OPERATIONAL, EVENTS.CREATE);
-
-    expectException();
-    stateMachine.getNextState(CREATING, EVENTS.CLOSE);
-  }
-
-  /**
-   * We expect an InvalidStateTransitionException.
-   */
-  private void expectException() {
-    exception.expect(InvalidStateTransitionException.class);
-    exception.expectMessage("Invalid event");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/package-info.java
new file mode 100644
index 0000000..1e9fd59
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.ozone.common;
+/**
+  ozone common shared by SCM, KSM, etc.
+ **/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
new file mode 100644
index 0000000..9aeff24
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/InvalidStateTransitionException.java
@@ -0,0 +1,42 @@
+/*
+ * 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.ozone.common.statemachine;
+
+/**
+ * Class wraps invalid state transition exception.
+ */
+public class InvalidStateTransitionException extends Exception {
+  private Enum<?> currentState;
+  private Enum<?> event;
+
+  public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
+    super("Invalid event: " + event + " at " + currentState + " state.");
+    this.currentState = currentState;
+    this.event = event;
+  }
+
+  public Enum<?> getCurrentState() {
+    return currentState;
+  }
+
+  public Enum<?> getEvent() {
+    return event;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
new file mode 100644
index 0000000..bf8cbd5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/StateMachine.java
@@ -0,0 +1,68 @@
+/*
+ * 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.ozone.common.statemachine;
+
+import com.google.common.base.Supplier;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Template class that wraps simple event driven state machine.
+ * @param <STATE> states allowed
+ * @param <EVENT> events allowed
+ */
+public class StateMachine<STATE extends Enum<?>, EVENT extends Enum<?>> {
+  private STATE initialState;
+  private Set<STATE> finalStates;
+
+  private final LoadingCache<EVENT, Map<STATE, STATE>> transitions =
+      CacheBuilder.newBuilder().build(
+          CacheLoader.from((Supplier<Map<STATE, STATE>>) () -> new HashMap()));
+
+  public StateMachine(STATE initState, Set<STATE> finalStates) {
+    this.initialState = initState;
+    this.finalStates = finalStates;
+  }
+
+  public STATE getInitialState() {
+    return initialState;
+  }
+
+  public Set<STATE> getFinalStates() {
+    return finalStates;
+  }
+
+  public STATE getNextState(STATE from, EVENT e)
+      throws InvalidStateTransitionException {
+    STATE target = transitions.getUnchecked(e).get(from);
+    if (target == null) {
+      throw new InvalidStateTransitionException(from, e);
+    }
+    return target;
+  }
+
+  public void addTransition(STATE from, STATE to, EVENT e) {
+    transitions.getUnchecked(e).put(from, to);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
new file mode 100644
index 0000000..045409e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/common/statemachine/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.ozone.common.statemachine;
+/**
+ state machine template class for ozone.
+ **/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c1e9ab2/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/common/TestStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/common/TestStateMachine.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/common/TestStateMachine.java
new file mode 100644
index 0000000..d4c626e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/ozone/common/TestStateMachine.java
@@ -0,0 +1,104 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.common;
+
+import org.apache.commons.collections.SetUtils;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.common.statemachine.StateMachine;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.INIT;
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.CREATING;
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.OPERATIONAL;
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.CLOSED;
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.CLEANUP;
+import static org.apache.hadoop.ozone.common.TestStateMachine.STATES.FINAL;
+
+/**
+ * This class is to test ozone common state machine.
+ */
+public class TestStateMachine {
+
+  /**
+   * STATES used by the test state machine.
+   */
+  public enum STATES {INIT, CREATING, OPERATIONAL, CLOSED, CLEANUP, FINAL};
+
+  /**
+   * EVENTS used by the test state machine.
+   */
+  public enum EVENTS {ALLOCATE, CREATE, UPDATE, CLOSE, DELETE, TIMEOUT};
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Test
+  public void testStateMachineStates() throws InvalidStateTransitionException {
+    Set<STATES> finals = new HashSet<>();
+    finals.add(FINAL);
+
+    StateMachine<STATES, EVENTS> stateMachine =
+        new StateMachine<>(INIT, finals);
+
+    stateMachine.addTransition(INIT, CREATING, EVENTS.ALLOCATE);
+    stateMachine.addTransition(CREATING, OPERATIONAL, EVENTS.CREATE);
+    stateMachine.addTransition(OPERATIONAL, OPERATIONAL, EVENTS.UPDATE);
+    stateMachine.addTransition(OPERATIONAL, CLEANUP, EVENTS.DELETE);
+    stateMachine.addTransition(OPERATIONAL, CLOSED, EVENTS.CLOSE);
+    stateMachine.addTransition(CREATING, CLEANUP, EVENTS.TIMEOUT);
+
+    // Initial and Final states
+    Assert.assertEquals("Initial State", INIT, stateMachine.getInitialState());
+    Assert.assertTrue("Final States", SetUtils.isEqualSet(finals,
+        stateMachine.getFinalStates()));
+
+    // Valid state transitions
+    Assert.assertEquals("STATE should be OPERATIONAL after being created",
+        OPERATIONAL, stateMachine.getNextState(CREATING, EVENTS.CREATE));
+    Assert.assertEquals("STATE should be OPERATIONAL after being updated",
+        OPERATIONAL, stateMachine.getNextState(OPERATIONAL, EVENTS.UPDATE));
+    Assert.assertEquals("STATE should be CLEANUP after being deleted",
+        CLEANUP, stateMachine.getNextState(OPERATIONAL, EVENTS.DELETE));
+    Assert.assertEquals("STATE should be CLEANUP after being timeout",
+        CLEANUP, stateMachine.getNextState(CREATING, EVENTS.TIMEOUT));
+    Assert.assertEquals("STATE should be CLOSED after being closed",
+        CLOSED, stateMachine.getNextState(OPERATIONAL, EVENTS.CLOSE));
+
+    // Negative cases: invalid transition
+    expectException();
+    stateMachine.getNextState(OPERATIONAL, EVENTS.CREATE);
+
+    expectException();
+    stateMachine.getNextState(CREATING, EVENTS.CLOSE);
+  }
+
+  /**
+   * We expect an InvalidStateTransitionException.
+   */
+  private void expectException() {
+    exception.expect(InvalidStateTransitionException.class);
+    exception.expectMessage("Invalid event");
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org