فهرست منبع

YARN-11283. Fix Typo of NodeManager amrmproxy. (#4899)

slfan1989 2 سال پیش
والد
کامیت
f52b900a5f
12فایلهای تغییر یافته به همراه88 افزوده شده و 88 حذف شده
  1. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContext.java
  2. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContextImpl.java
  3. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java
  4. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
  5. 14 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java
  6. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
  7. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/PassThroughRequestInterceptor.java
  8. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyMetrics.java
  9. 26 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
  10. 18 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
  11. 12 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptorSecure.java
  12. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContext.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 
 
 /**
 /**
- * Interface that can be used by the intercepter plugins to get the information
+ * Interface that can be used by the interceptor plugins to get the information
  * about one application.
  * about one application.
  *
  *
  */
  */

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContextImpl.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 
 
 /**
 /**
  * Encapsulates the information about one application that is needed by the
  * Encapsulates the information about one application that is needed by the
- * request intercepters.
+ * request interceptors.
  *
  *
  */
  */
 public class AMRMProxyApplicationContextImpl implements
 public class AMRMProxyApplicationContextImpl implements

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AbstractRequestInterceptor.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 
 
 /**
 /**
  * Implements the RequestInterceptor interface and provides common functionality
  * Implements the RequestInterceptor interface and provides common functionality
- * which can can be used and/or extended by other concrete intercepter classes.
+ * which can can be used and/or extended by other concrete interceptor classes.
  *
  *
  */
  */
 public abstract class AbstractRequestInterceptor implements
 public abstract class AbstractRequestInterceptor implements

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java

@@ -104,7 +104,7 @@ import org.apache.hadoop.util.Preconditions;
  * Extends the AbstractRequestInterceptor and provides an implementation for
  * Extends the AbstractRequestInterceptor and provides an implementation for
  * federation of YARN RM and scaling an application across multiple YARN
  * federation of YARN RM and scaling an application across multiple YARN
  * sub-clusters. All the federation specific implementation is encapsulated in
  * sub-clusters. All the federation specific implementation is encapsulated in
- * this class. This is always the last intercepter in the chain.
+ * this class. This is always the last interceptor in the chain.
  */
  */
 public class FederationInterceptor extends AbstractRequestInterceptor {
 public class FederationInterceptor extends AbstractRequestInterceptor {
   private static final Logger LOG =
   private static final Logger LOG =
@@ -654,7 +654,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       if (AMRMClientUtils.getNextResponseId(
       if (AMRMClientUtils.getNextResponseId(
           request.getResponseId()) == this.lastAllocateResponse
           request.getResponseId()) == this.lastAllocateResponse
               .getResponseId()) {
               .getResponseId()) {
-        // heartbeat one step old, simply return lastReponse
+        // heartbeat one step old, simply return lastResponse
         return this.lastAllocateResponse;
         return this.lastAllocateResponse;
       } else if (request.getResponseId() != this.lastAllocateResponse
       } else if (request.getResponseId() != this.lastAllocateResponse
           .getResponseId()) {
           .getResponseId()) {
@@ -1572,7 +1572,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   /**
   /**
    * Check to see if an AllocateRequest exists in the Map for the specified sub
    * Check to see if an AllocateRequest exists in the Map for the specified sub
    * cluster. If not found, create a new one, copy the value of responseId and
    * cluster. If not found, create a new one, copy the value of responseId and
-   * progress from the orignialAMRequest, save it in the specified Map and
+   * progress from the originalAMRequest, save it in the specified Map and
    * return the new instance. If found, just return the old instance.
    * return the new instance. If found, just return the old instance.
    */
    */
   private static AllocateRequest findOrCreateAllocateRequestForSubCluster(
   private static AllocateRequest findOrCreateAllocateRequestForSubCluster(

+ 14 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/RequestInterceptor.java

@@ -24,14 +24,14 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
 
 
 /**
 /**
- * Defines the contract to be implemented by the request intercepter classes,
+ * Defines the contract to be implemented by the request interceptor classes,
  * that can be used to intercept and inspect messages sent from the application
  * that can be used to intercept and inspect messages sent from the application
  * master to the resource manager.
  * master to the resource manager.
  */
  */
 public interface RequestInterceptor extends DistributedSchedulingAMProtocol,
 public interface RequestInterceptor extends DistributedSchedulingAMProtocol,
     Configurable {
     Configurable {
   /**
   /**
-   * This method is called for initializing the intercepter. This is guaranteed
+   * This method is called for initializing the interceptor. This is guaranteed
    * to be called only once in the lifetime of this instance.
    * to be called only once in the lifetime of this instance.
    *
    *
    * @param ctx AMRMProxy application context
    * @param ctx AMRMProxy application context
@@ -39,42 +39,42 @@ public interface RequestInterceptor extends DistributedSchedulingAMProtocol,
   void init(AMRMProxyApplicationContext ctx);
   void init(AMRMProxyApplicationContext ctx);
 
 
   /**
   /**
-   * Recover intercepter state when NM recovery is enabled. AMRMProxy will
+   * Recover interceptor state when NM recovery is enabled. AMRMProxy will
    * recover the data map into
    * recover the data map into
-   * AMRMProxyApplicationContext.getRecoveredDataMap(). All intercepters should
+   * AMRMProxyApplicationContext.getRecoveredDataMap(). All interceptors should
    * recover state from it.
    * recover state from it.
    *
    *
-   * For example, registerRequest has to be saved by the last intercepter (i.e.
+   * For example, registerRequest has to be saved by the last interceptor (i.e.
    * the one that actually connects to RM), in order to re-register when RM
    * the one that actually connects to RM), in order to re-register when RM
    * fails over.
    * fails over.
    *
    *
-   * @param recoveredDataMap states for all intercepters recovered from NMSS
+   * @param recoveredDataMap states for all interceptors recovered from NMSS
    */
    */
   void recover(Map<String, byte[]> recoveredDataMap);
   void recover(Map<String, byte[]> recoveredDataMap);
 
 
   /**
   /**
-   * This method is called to release the resources held by the intercepter.
+   * This method is called to release the resources held by the interceptor.
    * This will be called when the application pipeline is being destroyed. The
    * This will be called when the application pipeline is being destroyed. The
    * concrete implementations should dispose the resources and forward the
    * concrete implementations should dispose the resources and forward the
-   * request to the next intercepter, if any.
+   * request to the next interceptor, if any.
    */
    */
   void shutdown();
   void shutdown();
 
 
   /**
   /**
-   * Sets the next intercepter in the pipeline. The concrete implementation of
+   * Sets the next interceptor in the pipeline. The concrete implementation of
    * this interface should always pass the request to the nextInterceptor after
    * this interface should always pass the request to the nextInterceptor after
-   * inspecting the message. The last intercepter in the chain is responsible to
+   * inspecting the message. The last interceptor in the chain is responsible to
    * send the messages to the resource manager service and so the last
    * send the messages to the resource manager service and so the last
-   * intercepter will not receive this method call.
+   * interceptor will not receive this method call.
    *
    *
-   * @param nextInterceptor the next intercepter to set
+   * @param nextInterceptor the next interceptor to set
    */
    */
   void setNextInterceptor(RequestInterceptor nextInterceptor);
   void setNextInterceptor(RequestInterceptor nextInterceptor);
 
 
   /**
   /**
-   * Returns the next intercepter in the chain.
+   * Returns the next interceptor in the chain.
    * 
    * 
-   * @return the next intercepter in the chain
+   * @return the next interceptor in the chain
    */
    */
   RequestInterceptor getNextInterceptor();
   RequestInterceptor getNextInterceptor();
 
 

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java

@@ -134,7 +134,7 @@ public abstract class BaseAMRMProxyTest {
     String mockPassThroughInterceptorClass =
     String mockPassThroughInterceptorClass =
         PassThroughRequestInterceptor.class.getName();
         PassThroughRequestInterceptor.class.getName();
 
 
-    // Create a request intercepter pipeline for testing. The last one in the
+    // Create a request interceptor pipeline for testing. The last one in the
     // chain will call the mock resource manager. The others in the chain will
     // chain will call the mock resource manager. The others in the chain will
     // simply forward it to the next one in the chain
     // simply forward it to the next one in the chain
     config.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
     config.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
@@ -191,7 +191,7 @@ public abstract class BaseAMRMProxyTest {
     return new NMContext(null, null, null, null, stateStore, false, this.conf);
     return new NMContext(null, null, null, null, stateStore, false, this.conf);
   }
   }
 
 
-  // A utility method for intercepter recover unit test
+  // A utility method for interceptor recover unit test
   protected Map<String, byte[]> recoverDataMapForAppAttempt(
   protected Map<String, byte[]> recoverDataMapForAppAttempt(
       NMStateStoreService nmStateStore, ApplicationAttemptId attemptId)
       NMStateStoreService nmStateStore, ApplicationAttemptId attemptId)
       throws IOException {
       throws IOException {
@@ -341,7 +341,7 @@ public abstract class BaseAMRMProxyTest {
                   Assert.assertEquals(Integer.toString(index), response
                   Assert.assertEquals(Integer.toString(index), response
                       .getResponse().getQueue());
                       .getResponse().getQueue());
 
 
-                  LOG.info("Sucessfully registered application master with test context: "
+                  LOG.info("Successfully registered application master with test context: "
                       + testContext);
                       + testContext);
                 } catch (Throwable ex) {
                 } catch (Throwable ex) {
                   response = null;
                   response = null;
@@ -424,7 +424,7 @@ public abstract class BaseAMRMProxyTest {
                           testContext);
                           testContext);
                   Assert.assertNotNull(response.getResponse());
                   Assert.assertNotNull(response.getResponse());
 
 
-                  LOG.info("Sucessfully finished application master with test contexts: "
+                  LOG.info("Successfully finished application master with test contexts: "
                       + testContext);
                       + testContext);
                 } catch (Throwable ex) {
                 } catch (Throwable ex) {
                   response = null;
                   response = null;

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/PassThroughRequestInterceptor.java

@@ -29,8 +29,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 
 /**
 /**
- * Mock intercepter that does not do anything other than forwarding it to the
- * next intercepter in the chain
+ * Mock interceptor that does not do anything other than forwarding it to the
+ * next interceptor in the chain
  *
  *
  */
  */
 public class PassThroughRequestInterceptor extends
 public class PassThroughRequestInterceptor extends

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyMetrics.java

@@ -82,11 +82,11 @@ public class TestAMRMProxyMetrics extends BaseAMRMProxyTest {
     AllocateResponse allocateResponse = allocate(testAppId);
     AllocateResponse allocateResponse = allocate(testAppId);
     Assert.assertNotNull(allocateResponse);
     Assert.assertNotNull(allocateResponse);
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId, FinalApplicationStatus.SUCCEEDED);
         finishApplicationMaster(testAppId, FinalApplicationStatus.SUCCEEDED);
 
 
-    Assert.assertNotNull(finshResponse);
-    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+    Assert.assertNotNull(finishResponse);
+    Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
     Assert.assertEquals(failedAppStartRequests, metrics.getFailedAppStartRequests());
     Assert.assertEquals(failedAppStartRequests, metrics.getFailedAppStartRequests());
     Assert.assertEquals(failedRegisterAMRequests, metrics.getFailedRegisterAMRequests());
     Assert.assertEquals(failedRegisterAMRequests, metrics.getFailedRegisterAMRequests());
@@ -126,10 +126,10 @@ public class TestAMRMProxyMetrics extends BaseAMRMProxyTest {
     Assert
     Assert
         .assertEquals(Integer.toString(testAppId), registerResponse.getQueue());
         .assertEquals(Integer.toString(testAppId), registerResponse.getQueue());
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId, FinalApplicationStatus.FAILED);
         finishApplicationMaster(testAppId, FinalApplicationStatus.FAILED);
 
 
-    Assert.assertNotNull(finshResponse);
+    Assert.assertNotNull(finishResponse);
 
 
     try {
     try {
       // Try to finish an application master that is already finished.
       // Try to finish an application master that is already finished.

+ 26 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java

@@ -128,7 +128,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
           getAMRMProxyService().getPipelines();
           getAMRMProxyService().getPipelines();
       ApplicationId id = getApplicationId(testAppId);
       ApplicationId id = getApplicationId(testAppId);
       Assert.assertTrue(
       Assert.assertTrue(
-          "The interceptor pipeline should be removed if initializtion fails",
+          "The interceptor pipeline should be removed if initialization fails",
           pipelines.get(id) == null);
           pipelines.get(id) == null);
     }
     }
   }
   }
@@ -140,7 +140,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
    * @throws Exception
    * @throws Exception
    */
    */
   @Test
   @Test
-  public void testRegisterMulitpleApplicationMasters() throws Exception {
+  public void testRegisterMultipleApplicationMasters() throws Exception {
     for (int testAppId = 0; testAppId < 3; testAppId++) {
     for (int testAppId = 0; testAppId < 3; testAppId++) {
       RegisterApplicationMasterResponse response =
       RegisterApplicationMasterResponse response =
           registerApplicationMaster(testAppId);
           registerApplicationMaster(testAppId);
@@ -157,7 +157,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
    * @throws Exception
    * @throws Exception
    */
    */
   @Test
   @Test
-  public void testRegisterMulitpleApplicationMastersInParallel()
+  public void testRegisterMultipleApplicationMastersInParallel()
       throws Exception {
       throws Exception {
     int numberOfRequests = 5;
     int numberOfRequests = 5;
     ArrayList<String> testContexts =
     ArrayList<String> testContexts =
@@ -185,12 +185,12 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
     Assert.assertEquals(Integer.toString(testAppId),
     Assert.assertEquals(Integer.toString(testAppId),
         registerResponse.getQueue());
         registerResponse.getQueue());
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId,
         finishApplicationMaster(testAppId,
             FinalApplicationStatus.SUCCEEDED);
             FinalApplicationStatus.SUCCEEDED);
 
 
-    Assert.assertNotNull(finshResponse);
-    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+    Assert.assertNotNull(finishResponse);
+    Assert.assertEquals(true, finishResponse.getIsUnregistered());
   }
   }
 
 
   @Test
   @Test
@@ -202,10 +202,10 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
     Assert.assertEquals(Integer.toString(testAppId),
     Assert.assertEquals(Integer.toString(testAppId),
         registerResponse.getQueue());
         registerResponse.getQueue());
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId, FinalApplicationStatus.FAILED);
         finishApplicationMaster(testAppId, FinalApplicationStatus.FAILED);
 
 
-    Assert.assertNotNull(finshResponse);
+    Assert.assertNotNull(finishResponse);
 
 
     try {
     try {
       // Try to finish an application master that is already finished.
       // Try to finish an application master that is already finished.
@@ -232,7 +232,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
   }
   }
 
 
   @Test
   @Test
-  public void testFinishMulitpleApplicationMasters() throws Exception {
+  public void testFinishMultipleApplicationMasters() throws Exception {
     int numberOfRequests = 3;
     int numberOfRequests = 3;
     for (int index = 0; index < numberOfRequests; index++) {
     for (int index = 0; index < numberOfRequests; index++) {
       RegisterApplicationMasterResponse registerResponse =
       RegisterApplicationMasterResponse registerResponse =
@@ -244,11 +244,11 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
 
 
     // Finish in reverse sequence
     // Finish in reverse sequence
     for (int index = numberOfRequests - 1; index >= 0; index--) {
     for (int index = numberOfRequests - 1; index >= 0; index--) {
-      FinishApplicationMasterResponse finshResponse =
+      FinishApplicationMasterResponse finishResponse =
           finishApplicationMaster(index, FinalApplicationStatus.SUCCEEDED);
           finishApplicationMaster(index, FinalApplicationStatus.SUCCEEDED);
 
 
-      Assert.assertNotNull(finshResponse);
-      Assert.assertEquals(true, finshResponse.getIsUnregistered());
+      Assert.assertNotNull(finishResponse);
+      Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
       // Assert that the application has been removed from the collection
       // Assert that the application has been removed from the collection
       Assert.assertTrue(this.getAMRMProxyService()
       Assert.assertTrue(this.getAMRMProxyService()
@@ -277,7 +277,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
   }
   }
 
 
   @Test
   @Test
-  public void testFinishMulitpleApplicationMastersInParallel()
+  public void testFinishMultipleApplicationMastersInParallel()
       throws Exception {
       throws Exception {
     int numberOfRequests = 5;
     int numberOfRequests = 5;
     ArrayList<String> testContexts = new ArrayList<String>();
     ArrayList<String> testContexts = new ArrayList<String>();
@@ -308,12 +308,12 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
     AllocateResponse allocateResponse = allocate(testAppId);
     AllocateResponse allocateResponse = allocate(testAppId);
     Assert.assertNotNull(allocateResponse);
     Assert.assertNotNull(allocateResponse);
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId,
         finishApplicationMaster(testAppId,
             FinalApplicationStatus.SUCCEEDED);
             FinalApplicationStatus.SUCCEEDED);
 
 
-    Assert.assertNotNull(finshResponse);
-    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+    Assert.assertNotNull(finishResponse);
+    Assert.assertEquals(true, finishResponse.getIsUnregistered());
   }
   }
 
 
   @Test
   @Test
@@ -399,7 +399,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
                   getContainersAndAssert(testAppId, 10);
                   getContainersAndAssert(testAppId, 10);
               releaseContainersAndAssert(testAppId, containers);
               releaseContainersAndAssert(testAppId, containers);
 
 
-              LOG.info("Sucessfully registered application master with appId: "
+              LOG.info("Successfully registered application master with appId: "
                   + testAppId);
                   + testAppId);
             } catch (Throwable ex) {
             } catch (Throwable ex) {
               LOG.error(
               LOG.error(
@@ -533,7 +533,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
 
 
     // We need to make sure all the resource managers received the
     // We need to make sure all the resource managers received the
     // release list. The containers sent by the mock resource managers will be
     // release list. The containers sent by the mock resource managers will be
-    // aggregated and returned back to us and we can assert if all the release
+    // aggregated and returned back to us, and we can assert if all the release
     // lists reached the sub-clusters
     // lists reached the sub-clusters
     List<ContainerId> containersForReleasedContainerIds = new ArrayList<>();
     List<ContainerId> containersForReleasedContainerIds = new ArrayList<>();
     List<ContainerId> newlyFinished = getCompletedContainerIds(
     List<ContainerId> newlyFinished = getCompletedContainerIds(
@@ -607,19 +607,19 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
     allocateResponse = allocate(testAppId1);
     allocateResponse = allocate(testAppId1);
     Assert.assertNotNull(allocateResponse);
     Assert.assertNotNull(allocateResponse);
 
 
-    FinishApplicationMasterResponse finshResponse =
+    FinishApplicationMasterResponse finishResponse =
         finishApplicationMaster(testAppId1, FinalApplicationStatus.SUCCEEDED);
         finishApplicationMaster(testAppId1, FinalApplicationStatus.SUCCEEDED);
-    Assert.assertNotNull(finshResponse);
-    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+    Assert.assertNotNull(finishResponse);
+    Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
     allocateResponse = allocate(testAppId2);
     allocateResponse = allocate(testAppId2);
     Assert.assertNotNull(allocateResponse);
     Assert.assertNotNull(allocateResponse);
 
 
-    finshResponse =
+    finishResponse =
         finishApplicationMaster(testAppId2, FinalApplicationStatus.SUCCEEDED);
         finishApplicationMaster(testAppId2, FinalApplicationStatus.SUCCEEDED);
 
 
-    Assert.assertNotNull(finshResponse);
-    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+    Assert.assertNotNull(finishResponse);
+    Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
     int testAppId3 = 3;
     int testAppId3 = 3;
     try {
     try {
@@ -684,7 +684,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
   }
   }
 
 
   /**
   /**
-   * A mock intercepter implementation that uses the same mockRM instance across
+   * A mock interceptor implementation that uses the same mockRM instance across
    * restart.
    * restart.
    */
    */
   public static class MockRequestInterceptorAcrossRestart
   public static class MockRequestInterceptorAcrossRestart
@@ -723,7 +723,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
   }
   }
 
 
   /**
   /**
-   * A mock intercepter implementation that throws when recovering.
+   * A mock interceptor implementation that throws when recovering.
    */
    */
   public static class BadRequestInterceptorAcrossRestart
   public static class BadRequestInterceptorAcrossRestart
       extends MockRequestInterceptorAcrossRestart {
       extends MockRequestInterceptorAcrossRestart {

+ 18 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java

@@ -88,7 +88,7 @@ import org.slf4j.LoggerFactory;
  * Extends the TestAMRMProxyService and overrides methods in order to use the
  * Extends the TestAMRMProxyService and overrides methods in order to use the
  * AMRMProxyService's pipeline test cases for testing the FederationInterceptor
  * AMRMProxyService's pipeline test cases for testing the FederationInterceptor
  * class. The tests for AMRMProxyService has been written cleverly so that it
  * class. The tests for AMRMProxyService has been written cleverly so that it
- * can be reused to validate different request intercepter chains.
+ * can be reused to validate different request interceptor chains.
  */
  */
 public class TestFederationInterceptor extends BaseAMRMProxyTest {
 public class TestFederationInterceptor extends BaseAMRMProxyTest {
   private static final Logger LOG =
   private static final Logger LOG =
@@ -152,8 +152,8 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     String mockPassThroughInterceptorClass =
     String mockPassThroughInterceptorClass =
         PassThroughRequestInterceptor.class.getName();
         PassThroughRequestInterceptor.class.getName();
 
 
-    // Create a request intercepter pipeline for testing. The last one in the
-    // chain is the federation intercepter that calls the mock resource manager.
+    // Create a request interceptor pipeline for testing. The last one in the
+    // chain is the federation interceptor that calls the mock resource manager.
     // The others in the chain will simply forward it to the next one in the
     // The others in the chain will simply forward it to the next one in the
     // chain
     // chain
     conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
     conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
@@ -259,7 +259,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
 
 
     // The release request will be split and handled by the corresponding UAM.
     // The release request will be split and handled by the corresponding UAM.
     // The release containers returned by the mock resource managers will be
     // The release containers returned by the mock resource managers will be
-    // aggregated and returned back to us and we can check if total request size
+    // aggregated and returned back to us, and we can check if total request size
     // and returned size are the same
     // and returned size are the same
     List<ContainerId> containersForReleasedContainerIds =
     List<ContainerId> containersForReleasedContainerIds =
         new ArrayList<ContainerId>();
         new ArrayList<ContainerId>();
@@ -365,10 +365,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         finishReq.setTrackingUrl("");
         finishReq.setTrackingUrl("");
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
 
 
-        FinishApplicationMasterResponse finshResponse =
+        FinishApplicationMasterResponse finishResponse =
             interceptor.finishApplicationMaster(finishReq);
             interceptor.finishApplicationMaster(finishReq);
-        Assert.assertNotNull(finshResponse);
-        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        Assert.assertNotNull(finishResponse);
+        Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
         return null;
         return null;
       }
       }
@@ -425,10 +425,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         finishReq.setTrackingUrl("");
         finishReq.setTrackingUrl("");
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
 
 
-        FinishApplicationMasterResponse finshResponse =
+        FinishApplicationMasterResponse finishResponse =
             interceptor.finishApplicationMaster(finishReq);
             interceptor.finishApplicationMaster(finishReq);
-        Assert.assertNotNull(finshResponse);
-        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        Assert.assertNotNull(finishResponse);
+        Assert.assertEquals(true, finishResponse.getIsUnregistered());
         return null;
         return null;
       }
       }
     });
     });
@@ -573,7 +573,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
             interceptor.getSecondaryRMs();
             interceptor.getSecondaryRMs();
 
 
-        // Create a new intercepter instance and recover
+        // Create a new interceptor instance and recover
         interceptor = new TestableFederationInterceptor(homeRM, secondaries);
         interceptor = new TestableFederationInterceptor(homeRM, secondaries);
         interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
         interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
             getConf(), attemptId, "test-user", null, null, null, registryObj));
             getConf(), attemptId, "test-user", null, null, null, registryObj));
@@ -608,10 +608,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         finishReq.setTrackingUrl("");
         finishReq.setTrackingUrl("");
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
 
 
-        FinishApplicationMasterResponse finshResponse =
+        FinishApplicationMasterResponse finishResponse =
             interceptor.finishApplicationMaster(finishReq);
             interceptor.finishApplicationMaster(finishReq);
-        Assert.assertNotNull(finshResponse);
-        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        Assert.assertNotNull(finishResponse);
+        Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
         // After the application succeeds, the registry/NMSS entry should be
         // After the application succeeds, the registry/NMSS entry should be
         // cleaned up
         // cleaned up
@@ -849,7 +849,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
             interceptor.getSecondaryRMs();
             interceptor.getSecondaryRMs();
 
 
-        // Increase the attemptId and create a new intercepter instance for it
+        // Increase the attemptId and create a new interceptor instance for it
         attemptId = ApplicationAttemptId.newInstance(
         attemptId = ApplicationAttemptId.newInstance(
             attemptId.getApplicationId(), attemptId.getAttemptId() + 1);
             attemptId.getApplicationId(), attemptId.getAttemptId() + 1);
 
 
@@ -888,10 +888,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         finishReq.setTrackingUrl("");
         finishReq.setTrackingUrl("");
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
 
 
-        FinishApplicationMasterResponse finshResponse =
+        FinishApplicationMasterResponse finishResponse =
             interceptor.finishApplicationMaster(finishReq);
             interceptor.finishApplicationMaster(finishReq);
-        Assert.assertNotNull(finshResponse);
-        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        Assert.assertNotNull(finishResponse);
+        Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
         // After the application succeeds, the registry entry should be deleted
         // After the application succeeds, the registry entry should be deleted
         if (interceptor.getRegistryClient() != null) {
         if (interceptor.getRegistryClient() != null) {

+ 12 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptorSecure.java

@@ -102,7 +102,7 @@ public class TestFederationInterceptorSecure extends BaseAMRMProxyTest {
   private volatile int lastResponseId;
   private volatile int lastResponseId;
 
 
   private MockResourceManagerFacade mockHomeRm = null;
   private MockResourceManagerFacade mockHomeRm = null;
-  private Server homeClietRMRpcServer;
+  private Server homeClientRMRpcServer;
 
 
   private static File workDir;
   private static File workDir;
   private static final File TEST_ROOT_DIR =
   private static final File TEST_ROOT_DIR =
@@ -171,15 +171,15 @@ public class TestFederationInterceptorSecure extends BaseAMRMProxyTest {
 
 
   private void startRpcServer() {
   private void startRpcServer() {
     YarnRPC rpc = YarnRPC.create(conf);
     YarnRPC rpc = YarnRPC.create(conf);
-    this.homeClietRMRpcServer = rpc.getServer(ApplicationClientProtocol.class, mockHomeRm,
+    this.homeClientRMRpcServer = rpc.getServer(ApplicationClientProtocol.class, mockHomeRm,
         NetUtils.createSocketAddr(HOME_RM_ADDRESS), conf, null, 2);
         NetUtils.createSocketAddr(HOME_RM_ADDRESS), conf, null, 2);
-    this.homeClietRMRpcServer.start();
-    this.homeClietRMRpcServer.refreshServiceAcl(conf, new MockRMPolicyProvider());
+    this.homeClientRMRpcServer.start();
+    this.homeClientRMRpcServer.refreshServiceAcl(conf, new MockRMPolicyProvider());
   }
   }
 
 
   private void stopRpcServer() {
   private void stopRpcServer() {
-    if (homeClietRMRpcServer != null) {
-      homeClietRMRpcServer.stop();
+    if (homeClientRMRpcServer != null) {
+      homeClientRMRpcServer.stop();
     }
     }
   }
   }
 
 
@@ -200,8 +200,8 @@ public class TestFederationInterceptorSecure extends BaseAMRMProxyTest {
     String mockPassThroughInterceptorClass =
     String mockPassThroughInterceptorClass =
         PassThroughRequestInterceptor.class.getName();
         PassThroughRequestInterceptor.class.getName();
 
 
-    // Create a request intercepter pipeline for testing. The last one in the
-    // chain is the federation intercepter that calls the mock resource manager.
+    // Create a request interceptor pipeline for testing. The last one in the
+    // chain is the federation interceptor that calls the mock resource manager.
     // The others in the chain will simply forward it to the next one in the
     // The others in the chain will simply forward it to the next one in the
     // chain
     // chain
     conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
     conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
@@ -410,7 +410,7 @@ public class TestFederationInterceptorSecure extends BaseAMRMProxyTest {
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
         ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
             interceptor.getSecondaryRMs();
             interceptor.getSecondaryRMs();
 
 
-        // Create a new intercepter instance and recover
+        // Create a new interceptor instance and recover
         interceptor = new TestableFederationInterceptor(homeRM, secondaries);
         interceptor = new TestableFederationInterceptor(homeRM, secondaries);
         interceptor.init(new AMRMProxyApplicationContextImpl(nmContext, conf, attemptId,
         interceptor.init(new AMRMProxyApplicationContextImpl(nmContext, conf, attemptId,
             "test-user", null, null, null, registryObj));
             "test-user", null, null, null, registryObj));
@@ -445,10 +445,10 @@ public class TestFederationInterceptorSecure extends BaseAMRMProxyTest {
         finishReq.setTrackingUrl("");
         finishReq.setTrackingUrl("");
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
         finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
 
 
-        FinishApplicationMasterResponse finshResponse =
+        FinishApplicationMasterResponse finishResponse =
             interceptor.finishApplicationMaster(finishReq);
             interceptor.finishApplicationMaster(finishReq);
-        Assert.assertNotNull(finshResponse);
-        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        Assert.assertNotNull(finishResponse);
+        Assert.assertEquals(true, finishResponse.getIsUnregistered());
 
 
         // After the application succeeds, the registry/NMSS entry should be
         // After the application succeeds, the registry/NMSS entry should be
         // cleaned up
         // cleaned up

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java

@@ -258,7 +258,7 @@ public class TestableFederationInterceptor extends FederationInterceptor {
   }
   }
 
 
   /**
   /**
-   * Wrap the handler thread so it calls from the same user.
+   * Wrap the handler thread, so it calls from the same user.
    */
    */
   protected class TestableAMRequestHandlerThread
   protected class TestableAMRequestHandlerThread
       extends AMHeartbeatRequestHandler {
       extends AMHeartbeatRequestHandler {