Browse Source

YARN-3381. Fix typo InvalidStateTransitonException. Contributed by Brahma Reddy Battula.

(cherry picked from commit 19295b36d90e26616accee73b1f7743aab5df692)
Akira Ajisaka 10 years ago
parent
commit
0ba2defb9e
17 changed files with 90 additions and 45 deletions
  1. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  4. 3 0
      hadoop-yarn-project/CHANGES.txt
  5. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java
  6. 51 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java
  7. 6 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java
  8. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java
  9. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
  10. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
  11. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  12. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java
  13. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  14. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  15. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  16. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  17. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -122,7 +122,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -994,7 +994,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       JobStateInternal oldState = getInternalState();
       try {
          getStateMachine().doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         addDiagnostic("Invalid event " + event.getType() + 
             " on Job " + this.jobId);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -128,7 +128,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -1195,7 +1195,7 @@ public abstract class TaskAttemptImpl implements
       final TaskAttemptStateInternal oldState = getInternalState()  ;
       try {
         stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state for "
             + this.attemptId, e);
         eventHandler.handle(new JobDiagnosticsUpdateEvent(

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -85,7 +85,7 @@ import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -646,7 +646,7 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
       TaskStateInternal oldState = getInternalState();
       try {
         stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state for "
             + this.taskId, e);
         internalError(event.getType());

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -577,6 +577,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3894. RM startup should fail for wrong CS xml NodeLabel capacity
     configuration. (Bibin A Chundatt via wangda)
 
+    YARN-3381. Fix typo InvalidStateTransitonException.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/NMClientAsyncImpl.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -496,7 +496,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
       try {
         try {
           this.stateMachine.doTransition(event.getType(), event);
-        } catch (InvalidStateTransitonException e) {
+        } catch (InvalidStateTransitionException e) {
           LOG.error("Can't handle this event at current state", e);
         }
       } finally {

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitionException.java

@@ -0,0 +1,51 @@
+/**
+ * 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.yarn.state;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * The exception that happens when you call invalid state transition.
+ *
+ */
+@Public
+@Evolving
+public class InvalidStateTransitionException extends YarnRuntimeException {
+
+  private static final long serialVersionUID = -6188669113571351684L;
+  private Enum<?> currentState;
+  private Enum<?> event;
+
+  public InvalidStateTransitionException(Enum<?> currentState, Enum<?> event) {
+    super("Invalid event: " + event + " at " + currentState);
+    this.currentState = currentState;
+    this.event = event;
+  }
+
+  public Enum<?> getCurrentState() {
+    return currentState;
+  }
+
+  public Enum<?> getEvent() {
+    return event;
+  }
+
+}

+ 6 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/InvalidStateTransitonException.java

@@ -20,29 +20,20 @@ package org.apache.hadoop.yarn.state;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/** @deprecated Use {@link InvalidStateTransitionException} instead. */
 
 @Public
 @Evolving
-public class InvalidStateTransitonException extends YarnRuntimeException {
+@Deprecated
+public class InvalidStateTransitonException extends
+    InvalidStateTransitionException {
 
   private static final long serialVersionUID = 8610511635996283691L;
 
-  private Enum<?> currentState;
-  private Enum<?> event;
-
   public InvalidStateTransitonException(Enum<?> currentState, Enum<?> event) {
-    super("Invalid event: " + event + " at " + currentState);
-    this.currentState = currentState;
-    this.event = event;
+    super(currentState, event);
   }
 
-  public Enum<?> getCurrentState() {
-    return currentState;
-  }
-  
-  public Enum<?> getEvent() {
-    return event;
-  }
 
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachine.java

@@ -28,5 +28,5 @@ public interface StateMachine
                   EVENTTYPE extends Enum<EVENTTYPE>, EVENT> {
   public STATE getCurrentState();
   public STATE doTransition(EVENTTYPE eventType, EVENT event)
-        throws InvalidStateTransitonException;
+        throws InvalidStateTransitionException;
 }

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java

@@ -289,7 +289,7 @@ final public class StateMachineFactory
    */
   private STATE doTransition
            (OPERAND operand, STATE oldState, EVENTTYPE eventType, EVENT event)
-      throws InvalidStateTransitonException {
+      throws InvalidStateTransitionException {
     // We can assume that stateMachineTable is non-null because we call
     //  maybeMakeStateMachineTable() when we build an InnerStateMachine ,
     //  and this code only gets called from inside a working InnerStateMachine .
@@ -302,7 +302,7 @@ final public class StateMachineFactory
         return transition.doTransition(operand, oldState, event, eventType);
       }
     }
-    throw new InvalidStateTransitonException(oldState, eventType);
+    throw new InvalidStateTransitionException(oldState, eventType);
   }
 
   private synchronized void maybeMakeStateMachineTable() {
@@ -381,11 +381,11 @@ final public class StateMachineFactory
     @Override
     public STATE doTransition(OPERAND operand, STATE oldState,
                               EVENT event, EVENTTYPE eventType)
-        throws InvalidStateTransitonException {
+        throws InvalidStateTransitionException {
       STATE postState = hook.transition(operand, event);
 
       if (!validPostStates.contains(postState)) {
-        throw new InvalidStateTransitonException(oldState, eventType);
+        throw new InvalidStateTransitionException(oldState, eventType);
       }
       return postState;
     }
@@ -444,7 +444,7 @@ final public class StateMachineFactory
 
     @Override
     public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event)
-         throws InvalidStateTransitonException  {
+         throws InvalidStateTransitionException  {
       currentState = StateMachineFactory.this.doTransition
           (operand, currentState, eventType, event);
       return currentState;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java

@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -457,7 +457,7 @@ public class ApplicationImpl implements Application {
       try {
         // queue event requesting init of the same app
         newState = stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state", e);
       }
       if (oldState != newState) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -71,7 +71,7 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -1121,7 +1121,7 @@ public class ContainerImpl implements Container {
       try {
         newState =
             stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state: Current: ["
             + oldState + "], eventType: [" + event.getType() + "]", e);
       }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalizedResource.java

@@ -41,7 +41,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRecoveredEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceReleaseEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ResourceRequestEvent;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
@@ -196,7 +196,7 @@ public class LocalizedResource implements EventHandler<ResourceEvent> {
       ResourceState newState = null;
       try {
         newState = this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.warn("Can't handle this event at current state", e);
       }
       if (oldState != newState) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java

@@ -62,7 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppR
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
@@ -841,7 +841,7 @@ public abstract class RMStateStore extends AbstractService {
             + getRMStateStoreState());
       }
 
-    } catch (InvalidStateTransitonException e) {
+    } catch (InvalidStateTransitionException e) {
       LOG.error("Can't handle this event at current state", e);
     } finally {
       this.writeLock.unlock();

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -93,7 +93,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSch
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -759,7 +759,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       try {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -96,7 +96,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptA
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -784,7 +784,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       try {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -385,7 +385,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
       RMContainerState oldState = getState();
       try {
          stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         LOG.error("Invalid event " + event.getType() + 
             " on container " + this.containerId);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils.ContainerIdComparator;
-import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
@@ -446,7 +446,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       NodeState oldState = getState();
       try {
          stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
+      } catch (InvalidStateTransitionException e) {
         LOG.error("Can't handle this event at current state", e);
         LOG.error("Invalid event " + event.getType() + 
             " on Node  " + this.nodeId);