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 sh...@apache.org on 2018/08/26 00:44:13 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-15655. Enhance KMS client retry behavior. Contributed by Kitti Nanasi.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-12943 90e61bf0f -> 191faeb96


HADOOP-15655. Enhance KMS client retry behavior. Contributed by Kitti Nanasi.


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

Branch: refs/heads/HDFS-12943
Commit: edeb2a356ad671d962764c6e2aee9f9e7d6f394c
Parents: 2d13e41
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Aug 16 22:32:32 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Aug 16 22:42:03 2018 -0700

----------------------------------------------------------------------
 .../key/kms/LoadBalancingKMSClientProvider.java |  43 ++---
 .../kms/TestLoadBalancingKMSClientProvider.java | 181 ++++++++++++++++++-
 2 files changed, 193 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/edeb2a35/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
index 23cdc50..e68e844 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
@@ -113,8 +113,8 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     return providers;
   }
 
-  private <T> T doOp(ProviderCallable<T> op, int currPos)
-      throws IOException {
+  private <T> T doOp(ProviderCallable<T> op, int currPos,
+      boolean isIdempotent) throws IOException {
     if (providers.length == 0) {
       throw new IOException("No providers configured !");
     }
@@ -143,7 +143,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
         }
         RetryAction action = null;
         try {
-          action = retryPolicy.shouldRetry(ioe, 0, numFailovers, false);
+          action = retryPolicy.shouldRetry(ioe, 0, numFailovers, isIdempotent);
         } catch (Exception e) {
           if (e instanceof IOException) {
             throw (IOException)e;
@@ -201,7 +201,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public Token<?>[] call(KMSClientProvider provider) throws IOException {
         return provider.addDelegationTokens(renewer, credentials);
       }
-    }, nextIdx());
+    }, nextIdx(), false);
   }
 
   @Override
@@ -211,7 +211,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public Long call(KMSClientProvider provider) throws IOException {
         return provider.renewDelegationToken(token);
       }
-    }, nextIdx());
+    }, nextIdx(), false);
   }
 
   @Override
@@ -222,7 +222,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
         provider.cancelDelegationToken(token);
         return null;
       }
-    }, nextIdx());
+    }, nextIdx(), false);
   }
 
   // This request is sent to all providers in the load-balancing group
@@ -275,7 +275,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
             throws IOException, GeneralSecurityException {
           return provider.generateEncryptedKey(encryptionKeyName);
         }
-      }, nextIdx());
+      }, nextIdx(), true);
     } catch (WrapperException we) {
       if (we.getCause() instanceof GeneralSecurityException) {
         throw (GeneralSecurityException) we.getCause();
@@ -295,7 +295,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
             throws IOException, GeneralSecurityException {
           return provider.decryptEncryptedKey(encryptedKeyVersion);
         }
-      }, nextIdx());
+      }, nextIdx(), true);
     } catch (WrapperException we) {
       if (we.getCause() instanceof GeneralSecurityException) {
         throw (GeneralSecurityException) we.getCause();
@@ -315,7 +315,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
             throws IOException, GeneralSecurityException {
           return provider.reencryptEncryptedKey(ekv);
         }
-      }, nextIdx());
+      }, nextIdx(), true);
     } catch (WrapperException we) {
       if (we.getCause() instanceof GeneralSecurityException) {
         throw (GeneralSecurityException) we.getCause();
@@ -335,7 +335,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
           provider.reencryptEncryptedKeys(ekvs);
           return null;
         }
-      }, nextIdx());
+      }, nextIdx(), true);
     } catch (WrapperException we) {
       if (we.getCause() instanceof GeneralSecurityException) {
         throw (GeneralSecurityException) we.getCause();
@@ -351,7 +351,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public KeyVersion call(KMSClientProvider provider) throws IOException {
         return provider.getKeyVersion(versionName);
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
 
   @Override
@@ -361,7 +361,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public List<String> call(KMSClientProvider provider) throws IOException {
         return provider.getKeys();
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
 
   @Override
@@ -371,7 +371,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public Metadata[] call(KMSClientProvider provider) throws IOException {
         return provider.getKeysMetadata(names);
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
 
   @Override
@@ -382,7 +382,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
           throws IOException {
         return provider.getKeyVersions(name);
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
 
   @Override
@@ -392,8 +392,9 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public KeyVersion call(KMSClientProvider provider) throws IOException {
         return provider.getCurrentKey(name);
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
+
   @Override
   public Metadata getMetadata(final String name) throws IOException {
     return doOp(new ProviderCallable<Metadata>() {
@@ -401,7 +402,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public Metadata call(KMSClientProvider provider) throws IOException {
         return provider.getMetadata(name);
       }
-    }, nextIdx());
+    }, nextIdx(), true);
   }
 
   @Override
@@ -412,7 +413,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public KeyVersion call(KMSClientProvider provider) throws IOException {
         return provider.createKey(name, material, options);
       }
-    }, nextIdx());
+    }, nextIdx(), false);
   }
 
   @Override
@@ -425,7 +426,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
             NoSuchAlgorithmException {
           return provider.createKey(name, options);
         }
-      }, nextIdx());
+      }, nextIdx(), false);
     } catch (WrapperException e) {
       if (e.getCause() instanceof GeneralSecurityException) {
         throw (NoSuchAlgorithmException) e.getCause();
@@ -442,7 +443,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
         provider.deleteKey(name);
         return null;
       }
-    }, nextIdx());
+    }, nextIdx(), false);
   }
 
   @Override
@@ -453,7 +454,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
       public KeyVersion call(KMSClientProvider provider) throws IOException {
         return provider.rollNewVersion(name, material);
       }
-    }, nextIdx());
+    }, nextIdx(), false);
     invalidateCache(name);
     return newVersion;
   }
@@ -468,7 +469,7 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
             NoSuchAlgorithmException {
           return provider.rollNewVersion(name);
         }
-      }, nextIdx());
+      }, nextIdx(), false);
       invalidateCache(name);
       return newVersion;
     } catch (WrapperException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/edeb2a35/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
index 4e7aed9..058db92 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
@@ -29,10 +29,13 @@ import static org.mockito.Mockito.verify;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.NoRouteToHostException;
+import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
 
 import javax.net.ssl.SSLHandshakeException;
 
@@ -355,24 +358,27 @@ public class TestLoadBalancingKMSClientProvider {
   }
 
   /**
-   * Tests whether retryPolicy fails immediately, after trying each provider
-   * once, on encountering IOException which is not SocketException.
+   * Tests whether retryPolicy fails immediately on non-idempotent operations,
+   * after trying each provider once,
+   * on encountering IOException which is not SocketException.
    * @throws Exception
    */
   @Test
-  public void testClientRetriesWithIOException() throws Exception {
+  public void testClientRetriesNonIdempotentOpWithIOExceptionFailsImmediately()
+      throws Exception {
     Configuration conf = new Configuration();
+    final String keyName = "test";
     // Setting total failover attempts to .
     conf.setInt(
         CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 10);
     KMSClientProvider p1 = mock(KMSClientProvider.class);
-    when(p1.getMetadata(Mockito.anyString()))
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
         .thenThrow(new IOException("p1"));
     KMSClientProvider p2 = mock(KMSClientProvider.class);
-    when(p2.getMetadata(Mockito.anyString()))
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
         .thenThrow(new IOException("p2"));
     KMSClientProvider p3 = mock(KMSClientProvider.class);
-    when(p3.getMetadata(Mockito.anyString()))
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
         .thenThrow(new IOException("p3"));
 
     when(p1.getKMSUrl()).thenReturn("p1");
@@ -381,17 +387,61 @@ public class TestLoadBalancingKMSClientProvider {
     LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
         new KMSClientProvider[] {p1, p2, p3}, 0, conf);
     try {
-      kp.getMetadata("test3");
+      kp.createKey(keyName, new Options(conf));
       fail("Should fail since all providers threw an IOException");
     } catch (Exception e) {
       assertTrue(e instanceof IOException);
     }
     verify(kp.getProviders()[0], Mockito.times(1))
-        .getMetadata(Mockito.eq("test3"));
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+    verify(kp.getProviders()[1], Mockito.times(1))
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+    verify(kp.getProviders()[2], Mockito.times(1))
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+  }
+
+  /**
+   * Tests whether retryPolicy retries on idempotent operations
+   * when encountering IOException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesIdempotentOpWithIOExceptionSucceedsSecondTime()
+      throws Exception {
+    Configuration conf = new Configuration();
+    final String keyName = "test";
+    final KeyProvider.KeyVersion keyVersion
+        = new KMSClientProvider.KMSKeyVersion(keyName, "v1",
+        new byte[0]);
+    // Setting total failover attempts to .
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 10);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.getCurrentKey(Mockito.anyString()))
+        .thenThrow(new IOException("p1"))
+        .thenReturn(keyVersion);
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.getCurrentKey(Mockito.anyString()))
+        .thenThrow(new IOException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.getCurrentKey(Mockito.anyString()))
+        .thenThrow(new IOException("p3"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2, p3}, 0, conf);
+
+    KeyProvider.KeyVersion result = kp.getCurrentKey(keyName);
+
+    assertEquals(keyVersion, result);
+    verify(kp.getProviders()[0], Mockito.times(2))
+        .getCurrentKey(Mockito.eq(keyName));
     verify(kp.getProviders()[1], Mockito.times(1))
-        .getMetadata(Mockito.eq("test3"));
+        .getCurrentKey(Mockito.eq(keyName));
     verify(kp.getProviders()[2], Mockito.times(1))
-        .getMetadata(Mockito.eq("test3"));
+        .getCurrentKey(Mockito.eq(keyName));
   }
 
   /**
@@ -717,4 +767,115 @@ public class TestLoadBalancingKMSClientProvider {
     verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName),
         Mockito.any(Options.class));
   }
+
+  /**
+   * Tests that if an idempotent operation succeeds second time after
+   * SocketTimeoutException, then the operation is successful.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesIdempotentOpWithSocketTimeoutExceptionSucceeds()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3);
+    final List<String> keys = Arrays.asList("testKey");
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.getKeys())
+        .thenThrow(new SocketTimeoutException("p1"))
+        .thenReturn(keys);
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.getKeys()).thenThrow(new SocketTimeoutException("p2"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    List<String> result = kp.getKeys();
+    assertEquals(keys, result);
+    verify(p1, Mockito.times(2)).getKeys();
+    verify(p2, Mockito.times(1)).getKeys();
+  }
+
+  /**
+   * Tests that if a non idempotent operation fails at every attempt
+   * after SocketTimeoutException, then SocketTimeoutException is thrown.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesIdempotentOpWithSocketTimeoutExceptionFails()
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 2);
+    final String keyName = "test";
+    final String exceptionMessage = "p1 exception message";
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    Exception originalEx = new SocketTimeoutException(exceptionMessage);
+    when(p1.getKeyVersions(Mockito.anyString()))
+        .thenThrow(originalEx);
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.getKeyVersions(Mockito.anyString()))
+        .thenThrow(new SocketTimeoutException("p2 exception message"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2}, 0, conf);
+
+    Exception interceptedEx = intercept(SocketTimeoutException.class,
+        "SocketTimeoutException: " + exceptionMessage,
+        ()-> kp.getKeyVersions(keyName));
+    assertEquals(originalEx, interceptedEx);
+
+    verify(p1, Mockito.times(2))
+        .getKeyVersions(Mockito.eq(keyName));
+    verify(p2, Mockito.times(1))
+        .getKeyVersions(Mockito.eq(keyName));
+  }
+
+  /**
+   * Tests whether retryPolicy fails immediately on non-idempotent operations,
+   * after trying each provider once, on encountering SocketTimeoutException.
+   * @throws Exception
+   */
+  @Test
+  public void testClientRetriesNonIdempotentOpWithSocketTimeoutExceptionFails()
+      throws Exception {
+    Configuration conf = new Configuration();
+    final String keyName = "test";
+    // Setting total failover attempts to .
+    conf.setInt(
+        CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 10);
+    KMSClientProvider p1 = mock(KMSClientProvider.class);
+    when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new SocketTimeoutException("p1"));
+    KMSClientProvider p2 = mock(KMSClientProvider.class);
+    when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new SocketTimeoutException("p2"));
+    KMSClientProvider p3 = mock(KMSClientProvider.class);
+    when(p3.createKey(Mockito.anyString(), Mockito.any(Options.class)))
+        .thenThrow(new SocketTimeoutException("p3"));
+
+    when(p1.getKMSUrl()).thenReturn("p1");
+    when(p2.getKMSUrl()).thenReturn("p2");
+    when(p3.getKMSUrl()).thenReturn("p3");
+    LoadBalancingKMSClientProvider kp = new LoadBalancingKMSClientProvider(
+        new KMSClientProvider[] {p1, p2, p3}, 0, conf);
+    try {
+      kp.createKey(keyName, new Options(conf));
+      fail("Should fail since all providers threw a SocketTimeoutException");
+    } catch (Exception e) {
+      assertTrue(e instanceof SocketTimeoutException);
+    }
+    verify(kp.getProviders()[0], Mockito.times(1))
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+    verify(kp.getProviders()[1], Mockito.times(1))
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+    verify(kp.getProviders()[2], Mockito.times(1))
+        .createKey(Mockito.eq(keyName), Mockito.any(Options.class));
+  }
 }
\ No newline at end of file


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


[23/50] [abbrv] hadoop git commit: HDFS-13772. Erasure coding: Unnecessary NameNode Logs displaying for Enabling/Disabling Erasure coding policies which are already enabled/disabled. Contributed by Ayush Saxena

Posted by sh...@apache.org.
HDFS-13772. Erasure coding: Unnecessary NameNode Logs displaying for Enabling/Disabling Erasure coding policies which are already enabled/disabled. Contributed by Ayush Saxena


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

Branch: refs/heads/HDFS-12943
Commit: 770d9d9bb686bacef601ec8c4f884dc5afa9d4e2
Parents: 34577d2
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Aug 21 09:33:19 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Aug 21 09:33:19 2018 +0530

----------------------------------------------------------------------
 .../namenode/ErasureCodingPolicyManager.java    | 15 +++++++----
 .../server/namenode/FSDirErasureCodingOp.java   | 22 ++++++++++++-----
 .../hdfs/server/namenode/FSNamesystem.java      | 26 +++++++++++---------
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 +++---
 .../server/namenode/TestNamenodeRetryCache.java |  2 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |  2 +-
 6 files changed, 47 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 3a310da..e7de05b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -356,7 +356,7 @@ public final class ErasureCodingPolicyManager {
   /**
    * Disable an erasure coding policy by policyName.
    */
-  public synchronized void disablePolicy(String name) {
+  public synchronized boolean disablePolicy(String name) {
     ErasureCodingPolicyInfo info = policiesByName.get(name);
     if (info == null) {
       throw new HadoopIllegalArgumentException("The policy name " +
@@ -367,27 +367,32 @@ public final class ErasureCodingPolicyManager {
       enabledPoliciesByName.remove(name);
       enabledPolicies =
           enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
+      info.setState(ErasureCodingPolicyState.DISABLED);
+      LOG.info("Disable the erasure coding policy " + name);
+      return true;
     }
-    info.setState(ErasureCodingPolicyState.DISABLED);
-    LOG.info("Disable the erasure coding policy " + name);
+    return false;
   }
 
   /**
    * Enable an erasure coding policy by policyName.
    */
-  public synchronized void enablePolicy(String name) {
+  public synchronized boolean enablePolicy(String name) {
     final ErasureCodingPolicyInfo info = policiesByName.get(name);
     if (info == null) {
       throw new HadoopIllegalArgumentException("The policy name " +
           name + " does not exist");
     }
-
+    if (enabledPoliciesByName.containsKey(name)) {
+      return false;
+    }
     final ErasureCodingPolicy ecPolicy = info.getPolicy();
     enabledPoliciesByName.put(name, ecPolicy);
     info.setState(ErasureCodingPolicyState.ENABLED);
     enabledPolicies =
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 769c137..2ba840e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -253,11 +253,16 @@ final class FSDirErasureCodingOp {
    *                      rebuilding
    * @throws IOException
    */
-  static void enableErasureCodingPolicy(final FSNamesystem fsn,
+  static boolean enableErasureCodingPolicy(final FSNamesystem fsn,
       String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
-    fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
-    fsn.getEditLog().logEnableErasureCodingPolicy(ecPolicyName, logRetryCache);
+    boolean success =
+        fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
+    if (success) {
+      fsn.getEditLog().logEnableErasureCodingPolicy(ecPolicyName,
+          logRetryCache);
+    }
+    return success;
   }
 
   /**
@@ -269,11 +274,16 @@ final class FSDirErasureCodingOp {
    *                      rebuilding
    * @throws IOException
    */
-  static void disableErasureCodingPolicy(final FSNamesystem fsn,
+  static boolean disableErasureCodingPolicy(final FSNamesystem fsn,
       String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
-    fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
-    fsn.getEditLog().logDisableErasureCodingPolicy(ecPolicyName, logRetryCache);
+    boolean success =
+        fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
+    if (success) {
+      fsn.getEditLog().logDisableErasureCodingPolicy(ecPolicyName,
+          logRetryCache);
+    }
+    return success;
   }
 
   private static List<XAttr> removeErasureCodingPolicyXAttr(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index cdd7d48..b0fb26c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7604,29 +7604,31 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param ecPolicyName the name of the policy to be enabled
    * @param logRetryCache whether to record RPC ids in editlog for retry cache
    *                      rebuilding
+   * @return
    * @throws IOException
    */
-  void enableErasureCodingPolicy(String ecPolicyName,
+  boolean enableErasureCodingPolicy(String ecPolicyName,
       final boolean logRetryCache) throws IOException {
     final String operationName = "enableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
-    LOG.info("Enable the erasure coding policy " + ecPolicyName);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot enable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.enableErasureCodingPolicy(this, ecPolicyName,
-          logRetryCache);
-      success = true;
+      success = FSDirErasureCodingOp.enableErasureCodingPolicy(this,
+          ecPolicyName, logRetryCache);
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
+        logAuditEvent(success, operationName, ecPolicyName, null, null);
       }
-      logAuditEvent(success, operationName, ecPolicyName, null, null);
     }
+    return success;
   }
 
   /**
@@ -7636,7 +7638,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *                      rebuilding
    * @throws IOException
    */
-  void disableErasureCodingPolicy(String ecPolicyName,
+  boolean disableErasureCodingPolicy(String ecPolicyName,
       final boolean logRetryCache) throws IOException {
     final String operationName = "disableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
@@ -7647,16 +7649,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.disableErasureCodingPolicy(this, ecPolicyName,
-          logRetryCache);
-      success = true;
+      success = FSDirErasureCodingOp.disableErasureCodingPolicy(this,
+          ecPolicyName, logRetryCache);
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
+        logAuditEvent(success, operationName, ecPolicyName, null, null);
       }
-      logAuditEvent(success, operationName, ecPolicyName, null, null);
     }
+    return success;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 318f801..ec5ce9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2477,8 +2477,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.enableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
-      success = true;
+      success = namesystem.enableErasureCodingPolicy(ecPolicyName,
+          cacheEntry != null);
     } finally {
       RetryCache.setState(cacheEntry, success);
     }
@@ -2495,8 +2495,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
     boolean success = false;
     try {
-      namesystem.disableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
-      success = true;
+      success = namesystem.disableErasureCodingPolicy(ecPolicyName,
+          cacheEntry != null);
     } finally {
       RetryCache.setState(cacheEntry, success);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index 0995f13..a1946c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -463,7 +463,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 38, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/770d9d9b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index 43aaa92..bcdf511 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -194,7 +194,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 39, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 38, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();


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


[06/50] [abbrv] hadoop git commit: HDDS-218. add existing docker-compose files to the ozone release artifact. Contributed by Elek Marton.

Posted by sh...@apache.org.
HDDS-218. add existing docker-compose files to the ozone release artifact. Contributed by Elek Marton.


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

Branch: refs/heads/HDFS-12943
Commit: 9dd5d5ba713240c559b102fa3172b10077f5da87
Parents: fa121eb
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Fri Aug 17 07:58:24 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Fri Aug 17 07:58:24 2018 -0700

----------------------------------------------------------------------
 dev-support/bin/ozone-dist-layout-stitching     |  2 +
 hadoop-dist/pom.xml                             | 17 -------
 hadoop-dist/src/main/compose/README.md          | 51 ++++++++++++++++++++
 .../src/main/compose/ozone/docker-compose.yaml  |  6 +--
 .../main/compose/ozoneperf/docker-compose.yaml  |  6 +--
 hadoop-ozone/docs/content/GettingStarted.md     |  2 +-
 6 files changed, 60 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/dev-support/bin/ozone-dist-layout-stitching
----------------------------------------------------------------------
diff --git a/dev-support/bin/ozone-dist-layout-stitching b/dev-support/bin/ozone-dist-layout-stitching
index c30a37d..2ba7791 100755
--- a/dev-support/bin/ozone-dist-layout-stitching
+++ b/dev-support/bin/ozone-dist-layout-stitching
@@ -151,6 +151,8 @@ cp "${ROOT}/hadoop-ozone/ozonefs/target/hadoop-ozone-filesystem-${HDDS_VERSION}.
 cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ozoneManager/
 cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/hdds/webapps/scm/
 
+#Copy docker compose files
+run cp -p -r "${ROOT}/hadoop-dist/src/main/compose" .
 
 mkdir -p ./share/hadoop/mapreduce
 mkdir -p ./share/hadoop/yarn

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/hadoop-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index 5de6759..da05015 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -277,23 +277,6 @@
             <artifactId>maven-resources-plugin</artifactId>
             <executions>
               <execution>
-                <id>copy-docker-compose</id>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <phase>prepare-package</phase>
-                <configuration>
-                  <outputDirectory>${project.build.directory}/compose
-                  </outputDirectory>
-                  <resources>
-                    <resource>
-                      <directory>src/main/compose</directory>
-                      <filtering>true</filtering>
-                    </resource>
-                  </resources>
-                </configuration>
-              </execution>
-              <execution>
                 <id>copy-dockerfile</id>
                 <goals>
                   <goal>copy-resources</goal>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/hadoop-dist/src/main/compose/README.md
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/README.md b/hadoop-dist/src/main/compose/README.md
new file mode 100644
index 0000000..8189d2c
--- /dev/null
+++ b/hadoop-dist/src/main/compose/README.md
@@ -0,0 +1,51 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# Docker cluster definitions
+
+This directory contains multiple docker cluster definitions to start local pseudo cluster with different configuration.
+
+It helps to start local (multi-node like) pseudo cluster with docker and docker-compose and obviously it's not for production.
+
+You may find more information in the specific subdirectories but in generic you can use the following commands:
+
+## Usage
+
+To start a cluster go to a subdirectory and start the cluster:
+
+```
+docker-compose up -d
+```
+
+You can check the logs of all the components with:
+
+```
+docker-compose logs
+```
+
+In case of a problem you can destroy the cluster an delete all the local state with:
+
+```
+docker-compose down
+```
+
+(Note: a simple docker-compose stop may not delete all the local data).
+
+You can scale up and down the components:
+
+```
+docker-compose scale datanode=5
+```
+
+Usually the key webui ports are published on the docker host.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/hadoop-dist/src/main/compose/ozone/docker-compose.yaml
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozone/docker-compose.yaml b/hadoop-dist/src/main/compose/ozone/docker-compose.yaml
index bb5e8dd..0a6a9d8 100644
--- a/hadoop-dist/src/main/compose/ozone/docker-compose.yaml
+++ b/hadoop-dist/src/main/compose/ozone/docker-compose.yaml
@@ -19,7 +19,7 @@ services:
    datanode:
       image: apache/hadoop-runner
       volumes:
-        - ../../ozone:/opt/hadoop
+        - ../..:/opt/hadoop
       ports:
         - 9864
       command: ["/opt/hadoop/bin/ozone","datanode"]
@@ -28,7 +28,7 @@ services:
    ozoneManager:
       image: apache/hadoop-runner
       volumes:
-         - ../../ozone:/opt/hadoop
+         - ../..:/opt/hadoop
       ports:
          - 9874:9874
       environment:
@@ -39,7 +39,7 @@ services:
    scm:
       image: apache/hadoop-runner
       volumes:
-         - ../../ozone:/opt/hadoop
+         - ../..:/opt/hadoop
       ports:
          - 9876:9876
       env_file:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml b/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml
index 6d1d9ca..3b5cc7b 100644
--- a/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml
+++ b/hadoop-dist/src/main/compose/ozoneperf/docker-compose.yaml
@@ -19,7 +19,7 @@ services:
    datanode:
       image: apache/hadoop-runner
       volumes:
-        - ../../ozone:/opt/hadoop
+        - ../..:/opt/hadoop
         - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
         - 9864
@@ -29,7 +29,7 @@ services:
    ozoneManager:
       image: apache/hadoop-runner
       volumes:
-        - ../../ozone:/opt/hadoop
+        - ../..:/opt/hadoop
         - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
          - 9874:9874
@@ -41,7 +41,7 @@ services:
    scm:
       image: apache/hadoop-runner
       volumes:
-         - ../../ozone:/opt/hadoop
+         - ../..:/opt/hadoop
          - ./jmxpromo.jar:/opt/jmxpromo.jar
       ports:
          - 9876:9876

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9dd5d5ba/hadoop-ozone/docs/content/GettingStarted.md
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/content/GettingStarted.md b/hadoop-ozone/docs/content/GettingStarted.md
index 117a307..61d210a 100644
--- a/hadoop-ozone/docs/content/GettingStarted.md
+++ b/hadoop-ozone/docs/content/GettingStarted.md
@@ -81,7 +81,7 @@ Go to the directory where the docker compose files exist and tell
 `docker-compose` to start Ozone. This will start SCM, OM and a single datanode
 in the background.
 ```
-cd hadoop-dist/target/compose/ozone
+cd hadoop-dist/target/ozone/compose/ozone
 
 docker-compose up -d
 ```


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


[14/50] [abbrv] hadoop git commit: YARN-8679. [ATSv2] If HBase cluster is down for long time, high chances that NM ContainerManager dispatcher get blocked. Contributed by Wangda Tan.

Posted by sh...@apache.org.
YARN-8679. [ATSv2] If HBase cluster is down for long time, high chances that NM ContainerManager dispatcher get blocked. Contributed by Wangda Tan.


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

Branch: refs/heads/HDFS-12943
Commit: 4aacbfff605262aaf3dbd926258afcadc86c72c0
Parents: 79c97f6
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Sat Aug 18 10:26:55 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Sat Aug 18 10:26:55 2018 +0530

----------------------------------------------------------------------
 .../applicationsmanager/TestAMLaunchFailure.java       |  2 +-
 .../applicationsmanager/TestSchedulerNegotiator.java   |  2 +-
 .../TestTimelineServiceClientIntegration.java          |  3 ++-
 .../security/TestTimelineAuthFilterForV2.java          |  2 +-
 .../collector/PerNodeTimelineCollectorsAuxService.java | 13 +++++++++----
 5 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aacbfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
index c0009dd..ad39099 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
@@ -90,7 +90,7 @@ public class TestAMLaunchFailure {
 //    }
 //
 //    @Override
-//    public void addApplication(ApplicationId applicationId,
+//    public void addApplicationIfAbsent(ApplicationId applicationId,
 //        ApplicationMaster master, String user, String queue, Priority priority
 //        , ApplicationStore appStore)
 //        throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aacbfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
index 7d06e55..fedbf2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
@@ -67,7 +67,7 @@ public class TestSchedulerNegotiator {
 //      return null;
 //    }
 //    @Override
-//    public void addApplication(ApplicationId applicationId,
+//    public void addApplicationIfAbsent(ApplicationId applicationId,
 //        ApplicationMaster master, String user, String queue, Priority priority,
 //        ApplicationStore store)
 //        throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aacbfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
index 6a5ef55..7cf7428 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
@@ -81,7 +81,8 @@ public class TestTimelineServiceClientIntegration {
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
               collectorManager, conf);
-      auxService.addApplication(ApplicationId.newInstance(0, 1), "user");
+      auxService
+          .addApplicationIfAbsent(ApplicationId.newInstance(0, 1), "user");
     } catch (ExitUtil.ExitException e) {
       fail();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aacbfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
index bb511d8..356bfc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
@@ -210,7 +210,7 @@ public class TestTimelineAuthFilterForV2 {
           YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, "localhost");
     }
     ApplicationId appId = ApplicationId.newInstance(0, 1);
-    auxService.addApplication(
+    auxService.addApplicationIfAbsent(
         appId, UserGroupInformation.getCurrentUser().getUserName());
     if (!withKerberosLogin) {
       AppLevelTimelineCollector collector =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aacbfff/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
index c15f99d..82dd793 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
@@ -125,7 +125,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
    * @param user Application Master container user.
    * @return whether it was added successfully
    */
-  public boolean addApplication(ApplicationId appId, String user) {
+  public boolean addApplicationIfAbsent(ApplicationId appId, String user) {
     AppLevelTimelineCollector collector =
         new AppLevelTimelineCollectorWithAgg(appId, user);
     return (collectorManager.putIfAbsent(appId, collector)
@@ -156,15 +156,15 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
     if (context.getContainerType() == ContainerType.APPLICATION_MASTER) {
       ApplicationId appId = context.getContainerId().
           getApplicationAttemptId().getApplicationId();
-      synchronized (appIdToContainerId) {
+      synchronized (appIdToContainerId){
         Set<ContainerId> masterContainers = appIdToContainerId.get(appId);
         if (masterContainers == null) {
           masterContainers = new HashSet<>();
           appIdToContainerId.put(appId, masterContainers);
         }
         masterContainers.add(context.getContainerId());
-        addApplication(appId, context.getUser());
       }
+      addApplicationIfAbsent(appId, context.getUser());
     }
   }
 
@@ -189,6 +189,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
         containerId.getApplicationAttemptId().getApplicationId();
     return scheduler.schedule(new Runnable() {
       public void run() {
+        boolean shouldRemoveApplication = false;
         synchronized (appIdToContainerId) {
           Set<ContainerId> masterContainers = appIdToContainerId.get(appId);
           if (masterContainers == null) {
@@ -199,10 +200,14 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
           masterContainers.remove(containerId);
           if (masterContainers.size() == 0) {
             // remove only if it is last master container
-            removeApplication(appId);
+            shouldRemoveApplication = true;
             appIdToContainerId.remove(appId);
           }
         }
+
+        if (shouldRemoveApplication) {
+          removeApplication(appId);
+        }
       }
     }, collectorLingerPeriod, TimeUnit.MILLISECONDS);
   }


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


[10/50] [abbrv] hadoop git commit: HDFS-13822. speedup libhdfs++ build (enable parallel build). Contributed by Allen Wittenauer and Pradeep Ambati

Posted by sh...@apache.org.
HDFS-13822. speedup libhdfs++ build (enable parallel build). Contributed by Allen Wittenauer and Pradeep Ambati


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

Branch: refs/heads/HDFS-12943
Commit: a17eed1b870ede9c6519b260e2dfe721b270bdbb
Parents: d723285
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Aug 17 12:23:18 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Aug 17 12:25:36 2018 -0500

----------------------------------------------------------------------
 .../hadoop-common/HadoopJNI.cmake               |   2 +
 .../hadoop-hdfs-native-client/pom.xml           | 105 ++++++++++---------
 .../src/CMakeLists.txt                          |  48 +++++++++
 .../src/main/native/libhdfs/CMakeLists.txt      |   6 +-
 .../main/native/libhdfspp/tests/CMakeLists.txt  |   4 +
 5 files changed, 114 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a17eed1b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopJNI.cmake b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
index 78d7ffd..bf0d73e 100644
--- a/hadoop-common-project/hadoop-common/HadoopJNI.cmake
+++ b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
@@ -93,5 +93,7 @@ if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
 # Otherwise, use the standard FindJNI module to locate the JNI components.
 #
 else()
+    find_package(Java REQUIRED)
+    include(UseJava)
     find_package(JNI REQUIRED)
 endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a17eed1b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
index 52d9257..ded1c0d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
@@ -201,26 +201,36 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <build>
         <plugins>
           <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
             <executions>
               <execution>
-                <id>make</id>
+                <id>cmake-compile</id>
                 <phase>compile</phase>
-                <goals><goal>run</goal></goals>
+                <goals><goal>cmake-compile</goal></goals>
                 <configuration>
-                  <target>
-                    <mkdir dir="${project.build.directory}"/>
-                    <exec executable="cmake" dir="${project.build.directory}" failonerror="true">
-                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model}  -DHADOOP_BUILD=1 -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse} -DREQUIRE_VALGRIND=${require.valgrind} "/>
-                      <arg line="${native_cmake_args}"/>
-                    </exec>
-                    <exec executable="make" dir="${project.build.directory}" failonerror="true">
-                      <arg line="${native_make_args}"/>
-                    </exec>
-                  </target>
+                  <source>${basedir}/src</source>
+                  <vars>
+                    <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                    <JVM_ARCH_DATA_MODEL>${sun.arch.data.model}</JVM_ARCH_DATA_MODEL>
+                    <REQUIRE_FUSE>${require.fuse}</REQUIRE_FUSE>
+                    <REQUIRE_VALGRIND>${require.valgrind}</REQUIRE_VALGRIND>
+                    <HADOOP_BUILD>1</HADOOP_BUILD>
+                    <REQUIRE_LIBWEBHDFS>${require.libwebhdfs}</REQUIRE_LIBWEBHDFS>
+                    <REQUIRE_OPENSSL>${require.openssl}</REQUIRE_OPENSSL>
+                    <CUSTOM_OPENSSL_PREFIX>${openssl.prefix}</CUSTOM_OPENSSL_PREFIX>
+                    <CUSTOM_OPENSSL_LIB>${openssl.lib}</CUSTOM_OPENSSL_LIB>
+                    <CUSTOM_OPENSSL_INCLUDE>${openssl.include}</CUSTOM_OPENSSL_INCLUDE>
+                  </vars>
+                  <output>${project.build.directory}</output>
                 </configuration>
               </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
               <execution>
                 <id>native_tests</id>
                 <phase>test</phase>
@@ -236,6 +246,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                       <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
                       <!-- Make sure libhadoop.so is on LD_LIBRARY_PATH. -->
                       <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
+                      <env key="DYLD_LIBRARY_PATH" value="${env.DYLD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
                     </exec>
                   </target>
                 </configuration>
@@ -246,7 +257,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       </build>
     </profile>
     <profile>
-      <id>test-patch</id>
+      <id>native-clang</id>
       <activation>
         <activeByDefault>false</activeByDefault>
       </activation>
@@ -256,35 +267,40 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <build>
         <plugins>
           <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
             <executions>
               <execution>
-                <id>make_altern</id>
+                <id>cmake-compile-clang</id>
                 <phase>compile</phase>
-                <goals><goal>run</goal></goals>
+                <goals><goal>cmake-compile</goal></goals>
                 <configuration>
-                  <target>
-                    <mkdir dir="${project.build.directory}/altern"/>
-                    <condition property="c_compiler" value="clang" else="gcc">
-                      <contains string="${env.CC}" substring="gcc"/>
-                    </condition>
-                    <condition property="cxx_compiler" value="clang++" else="g++">
-                      <contains string="${env.CXX}" substring="g++"/>
-                    </condition>
-                    <exec executable="cmake" dir="${project.build.directory}/altern" failonerror="true">
-                      <arg line="${basedir}/src/ -DGENERATED_JAVAH=${project.build.directory}/altern/native/javah -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model}  -DHADOOP_BUILD=1 -DREQUIRE_LIBWEBHDFS=${require.libwebhdfs} -DREQUIRE_FUSE=${require.fuse} -DREQUIRE_VALGRIND=${require.valgrind} "/>
-                      <arg line="-DCMAKE_C_COMPILER=${c_compiler} -DCMAKE_CXX_COMPILER=${cxx_compiler}"/>
-                      <arg line="${native_cmake_args}"/>
-                    </exec>
-                    <exec executable="make" dir="${project.build.directory}/altern" failonerror="true">
-                      <arg line="${native_make_args}"/>
-                    </exec>
-                  </target>
+                  <source>${basedir}/src</source>
+                  <vars>
+                    <CMAKE_C_COMPILER>clang</CMAKE_C_COMPILER>
+                    <CMAKE_CXX_COMPILER>clang++</CMAKE_CXX_COMPILER>
+                    <GENERATED_JAVAH>${project.build.directory}/native/javah</GENERATED_JAVAH>
+                    <JVM_ARCH_DATA_MODEL>${sun.arch.data.model}</JVM_ARCH_DATA_MODEL>
+                    <REQUIRE_FUSE>${require.fuse}</REQUIRE_FUSE>
+                    <REQUIRE_VALGRIND>${require.valgrind}</REQUIRE_VALGRIND>
+                    <HADOOP_BUILD>1</HADOOP_BUILD>
+                    <REQUIRE_LIBWEBHDFS>${require.libwebhdfs}</REQUIRE_LIBWEBHDFS>
+                    <REQUIRE_OPENSSL>${require.openssl}</REQUIRE_OPENSSL>
+                    <CUSTOM_OPENSSL_PREFIX>${openssl.prefix}</CUSTOM_OPENSSL_PREFIX>
+                    <CUSTOM_OPENSSL_LIB>${openssl.lib}</CUSTOM_OPENSSL_LIB>
+                    <CUSTOM_OPENSSL_INCLUDE>${openssl.include}</CUSTOM_OPENSSL_INCLUDE>
+                  </vars>
+                  <output>${project.build.directory}/clang</output>
                 </configuration>
               </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
               <execution>
-                <id>native_tests_altern</id>
+                <id>native_tests_clang</id>
                 <phase>test</phase>
                 <goals><goal>run</goal></goals>
                 <configuration>
@@ -292,26 +308,17 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                   <target>
                     <property name="compile_classpath" refid="maven.compile.classpath"/>
                     <property name="test_classpath" refid="maven.test.classpath"/>
-                    <exec executable="ctest" failonerror="true" dir="${project.build.directory}/altern">
+                    <exec executable="ctest" failonerror="true" dir="${project.build.directory}/clang">
                       <arg line="--output-on-failure"/>
                       <arg line="${native_ctest_args}"/>
                       <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
                       <!-- Make sure libhadoop.so is on LD_LIBRARY_PATH. -->
-                      <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/altern/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
+                      <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/clang/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
+                      <env key="DYLD_LIBRARY_PATH" value="${env.DYLD_LIBRARY_PATH}:${project.build.directory}/clang/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
                     </exec>
                   </target>
                 </configuration>
               </execution>
-              <execution>
-                <id>clean_altern</id>
-                <phase>test</phase>
-                <goals><goal>run</goal></goals>
-                <configuration>
-                  <target>
-                    <delete dir="${project.build.directory}/altern" includeemptydirs="true"/>
-                  </target>
-                </configuration>
-              </execution>
             </executions>
           </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a17eed1b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
index a3f8f2d..1813ec1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
@@ -88,6 +88,54 @@ function(link_libhdfs_test NAME LIBRARY)
 target_link_libraries("${NAME}_${LIBRARY}" ${LIBRARY} ${ARGN})
 endfunction()
 
+
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_without_version()
+set(OPENSSL_NAME "crypto")
+if(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+    SET(OPENSSL_NAME "eay32")
+endif()
+message("CUSTOM_OPENSSL_PREFIX = ${CUSTOM_OPENSSL_PREFIX}")
+find_library(OPENSSL_LIBRARY
+    NAMES ${OPENSSL_NAME}
+    PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/lib
+          ${CUSTOM_OPENSSL_PREFIX}/lib64 ${CUSTOM_OPENSSL_LIB} NO_DEFAULT_PATH)
+find_library(OPENSSL_LIBRARY NAMES ${OPENSSL_NAME})
+find_path(OPENSSL_INCLUDE_DIR
+    NAMES openssl/evp.h
+    PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/include
+          ${CUSTOM_OPENSSL_INCLUDE} NO_DEFAULT_PATH)
+find_path(OPENSSL_INCLUDE_DIR NAMES openssl/evp.h)
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+set(USABLE_OPENSSL 0)
+if(OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
+    include(CheckCSourceCompiles)
+    set(OLD_CMAKE_REQUIRED_INCLUDES ${CMAKE_REQUIRED_INCLUDES})
+    set(CMAKE_REQUIRED_INCLUDES ${OPENSSL_INCLUDE_DIR})
+    check_c_source_compiles("#include \"${OPENSSL_INCLUDE_DIR}/openssl/evp.h\"\nint main(int argc, char **argv) { return !EVP_aes_256_ctr; }" HAS_NEW_ENOUGH_OPENSSL)
+    set(CMAKE_REQUIRED_INCLUDES ${OLD_CMAKE_REQUIRED_INCLUDES})
+    if(NOT HAS_NEW_ENOUGH_OPENSSL)
+        message("The OpenSSL library installed at ${OPENSSL_LIBRARY} is too old.  You need a version at least new enough to have EVP_aes_256_ctr.")
+    else()
+        SET(USABLE_OPENSSL 1)
+    endif()
+endif()
+if(USABLE_OPENSSL)
+    get_filename_component(HADOOP_OPENSSL_LIBRARY ${OPENSSL_LIBRARY} NAME)
+    set(OPENSSL_SOURCE_FILES
+        "${SRC}/crypto/OpensslCipher.c"
+        "${SRC}/crypto/random/OpensslSecureRandom.c")
+    set(REQUIRE_OPENSSL ${REQUIRE_OPENSSL}) # Stop warning about unused variable.
+else()
+    message("Cannot find a usable OpenSSL library. OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
+    if(REQUIRE_OPENSSL)
+        message(FATAL_ERROR "Terminating build because require.openssl was specified.")
+    endif()
+    set(OPENSSL_LIBRARY "")
+    set(OPENSSL_INCLUDE_DIR "")
+    set(OPENSSL_SOURCE_FILES "")
+endif()
+
 add_subdirectory(main/native/libhdfs)
 add_subdirectory(main/native/libhdfs-tests)
 add_subdirectory(main/native/libhdfspp)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a17eed1b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
index 2883585..cac1335 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
@@ -74,9 +74,11 @@ endif()
 # introducing an abstraction layer over the sys/mman.h functions.
 if(NOT WIN32)
     build_libhdfs_test(test_libhdfs_vecsum hdfs vecsum.c)
+    set(THREADS_PREFER_PTHREAD_FLAG ON)
+    find_package(Threads REQUIRED)
     if(CMAKE_SYSTEM_NAME MATCHES "Darwin")
-        link_libhdfs_test(test_libhdfs_vecsum hdfs pthread)
+        link_libhdfs_test(test_libhdfs_vecsum hdfs ${CMAKE_THREAD_LIBS_INIT})
     else()
-        link_libhdfs_test(test_libhdfs_vecsum hdfs pthread rt)
+        link_libhdfs_test(test_libhdfs_vecsum hdfs ${CMAKE_THREAD_LIBS_INIT} rt)
     endif()
 endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a17eed1b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
index 6157902..59fdbf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
@@ -143,6 +143,10 @@ include_directories (
 add_library(hdfspp_test_shim_static STATIC hdfs_shim.c libhdfs_wrapper.c libhdfspp_wrapper.cc ${LIBHDFSPP_BINDING_C}/hdfs.cc)
 add_library(hdfspp_test_static STATIC ${LIBHDFSPP_BINDING_C}/hdfs.cc)
 
+# Add dependencies
+add_dependencies(hdfspp_test_shim_static proto)
+add_dependencies(hdfspp_test_static proto)
+
 # TODO: get all of the mini dfs library bits here in one place
 # add_library(hdfspp_mini_cluster     native_mini_dfs ${JAVA_JVM_LIBRARY} )
 


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


[19/50] [abbrv] hadoop git commit: YARN-8242. YARN NM: OOM error while reading back the state store on recovery. Contributed by Pradeep Ambati and Kanwaljeet Sachdev

Posted by sh...@apache.org.
YARN-8242. YARN NM: OOM error while reading back the state store on recovery. Contributed by Pradeep Ambati and Kanwaljeet Sachdev


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

Branch: refs/heads/HDFS-12943
Commit: 65e7469712be6cf393e29ef73cc94727eec81227
Parents: 01ff817
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Aug 20 10:14:40 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Aug 20 10:14:40 2018 -0500

----------------------------------------------------------------------
 .../server/nodemanager/DeletionService.java     |  25 +-
 .../containermanager/ContainerManagerImpl.java  |  26 +-
 .../localizer/ResourceLocalizationService.java  |  56 +--
 .../recovery/NMLeveldbStateStoreService.java    | 412 ++++++++++++-------
 .../recovery/NMNullStateStoreService.java       |   2 +-
 .../recovery/NMStateStoreService.java           |  55 +--
 .../nodemanager/recovery/RecoveryIterator.java  |  41 ++
 .../security/NMContainerTokenSecretManager.java |  27 +-
 .../security/NMTokenSecretManagerInNM.java      |  15 +-
 .../recovery/NMMemoryStateStoreService.java     |  82 +++-
 .../TestNMLeveldbStateStoreService.java         | 216 +++++++---
 11 files changed, 647 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
index ae81dc1..e665c5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
@@ -19,13 +19,14 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -96,16 +97,20 @@ public class DeletionService extends AbstractService {
 
   private void recover(NMStateStoreService.RecoveredDeletionServiceState state)
       throws IOException {
-    List<DeletionServiceDeleteTaskProto> taskProtos = state.getTasks();
     Map<Integer, DeletionTaskRecoveryInfo> idToInfoMap =
-        new HashMap<>(taskProtos.size());
-    Set<Integer> successorTasks = new HashSet<>();
-    for (DeletionServiceDeleteTaskProto proto : taskProtos) {
-      DeletionTaskRecoveryInfo info =
-          NMProtoUtils.convertProtoToDeletionTaskRecoveryInfo(proto, this);
-      idToInfoMap.put(info.getTask().getTaskId(), info);
-      nextTaskId.set(Math.max(nextTaskId.get(), info.getTask().getTaskId()));
-      successorTasks.addAll(info.getSuccessorTaskIds());
+        new HashMap<Integer, DeletionTaskRecoveryInfo>();
+    Set<Integer> successorTasks = new HashSet<Integer>();
+
+    try (RecoveryIterator<DeletionServiceDeleteTaskProto> it =
+             state.getIterator()) {
+      while (it.hasNext()) {
+        DeletionServiceDeleteTaskProto proto = it.next();
+        DeletionTaskRecoveryInfo info =
+            NMProtoUtils.convertProtoToDeletionTaskRecoveryInfo(proto, this);
+        idToInfoMap.put(info.getTask().getTaskId(), info);
+        nextTaskId.set(Math.max(nextTaskId.get(), info.getTask().getTaskId()));
+        successorTasks.addAll(info.getSuccessorTaskIds());
+      }
     }
 
     // restore the task dependencies and schedule the deletion tasks that

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 8b35258..b89e2dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -23,6 +23,7 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerTokenUpdatedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -356,19 +357,26 @@ public class ContainerManagerImpl extends CompositeService implements
           stateStore.loadLocalizationState());
 
       RecoveredApplicationsState appsState = stateStore.loadApplicationsState();
-      for (ContainerManagerApplicationProto proto :
-           appsState.getApplications()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Recovering application with state: " + proto.toString());
+      try (RecoveryIterator<ContainerManagerApplicationProto> rasIterator =
+               appsState.getIterator()) {
+        while (rasIterator.hasNext()) {
+          ContainerManagerApplicationProto proto = rasIterator.next();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Recovering application with state: " + proto.toString());
+          }
+          recoverApplication(proto);
         }
-        recoverApplication(proto);
       }
 
-      for (RecoveredContainerState rcs : stateStore.loadContainersState()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Recovering container with state: " + rcs);
+      try (RecoveryIterator<RecoveredContainerState> rcsIterator =
+               stateStore.getContainerStateIterator()) {
+        while (rcsIterator.hasNext()) {
+          RecoveredContainerState rcs = rcsIterator.next();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Recovering container with state: " + rcs);
+          }
+          recoverContainer(rcs);
         }
-        recoverContainer(rcs);
       }
 
       // Recovery AMRMProxy state after apps and containers are recovered

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 3834ece..2892d1f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -295,42 +297,46 @@ public class ResourceLocalizationService extends CompositeService
 
   //Recover localized resources after an NM restart
   public void recoverLocalizedResources(RecoveredLocalizationState state)
-      throws URISyntaxException {
+      throws URISyntaxException, IOException {
     LocalResourceTrackerState trackerState = state.getPublicTrackerState();
     recoverTrackerResources(publicRsrc, trackerState);
 
-    for (Map.Entry<String, RecoveredUserResources> userEntry :
-         state.getUserResources().entrySet()) {
-      String user = userEntry.getKey();
-      RecoveredUserResources userResources = userEntry.getValue();
-      trackerState = userResources.getPrivateTrackerState();
-      if (!trackerState.isEmpty()) {
-        LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-            null, dispatcher, true, super.getConfig(), stateStore, dirsHandler);
-        LocalResourcesTracker oldTracker = privateRsrc.putIfAbsent(user,
-            tracker);
-        if (oldTracker != null) {
-          tracker = oldTracker;
-        }
-        recoverTrackerResources(tracker, trackerState);
-      }
-
-      for (Map.Entry<ApplicationId, LocalResourceTrackerState> appEntry :
-           userResources.getAppTrackerStates().entrySet()) {
-        trackerState = appEntry.getValue();
+    try (RecoveryIterator<Map.Entry<String, RecoveredUserResources>> it
+             = state.getIterator()) {
+      while (it.hasNext()) {
+        Map.Entry<String, RecoveredUserResources> userEntry = it.next();
+        String user = userEntry.getKey();
+        RecoveredUserResources userResources = userEntry.getValue();
+        trackerState = userResources.getPrivateTrackerState();
         if (!trackerState.isEmpty()) {
-          ApplicationId appId = appEntry.getKey();
-          String appIdStr = appId.toString();
           LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
-              appId, dispatcher, false, super.getConfig(), stateStore,
+              null, dispatcher, true, super.getConfig(), stateStore,
               dirsHandler);
-          LocalResourcesTracker oldTracker = appRsrc.putIfAbsent(appIdStr,
+          LocalResourcesTracker oldTracker = privateRsrc.putIfAbsent(user,
               tracker);
           if (oldTracker != null) {
             tracker = oldTracker;
           }
           recoverTrackerResources(tracker, trackerState);
         }
+
+        for (Map.Entry<ApplicationId, LocalResourceTrackerState> appEntry :
+            userResources.getAppTrackerStates().entrySet()) {
+          trackerState = appEntry.getValue();
+          if (!trackerState.isEmpty()) {
+            ApplicationId appId = appEntry.getKey();
+            String appIdStr = appId.toString();
+            LocalResourcesTracker tracker = new LocalResourcesTrackerImpl(user,
+                appId, dispatcher, false, super.getConfig(), stateStore,
+                dirsHandler);
+            LocalResourcesTracker oldTracker = appRsrc.putIfAbsent(appIdStr,
+                tracker);
+            if (oldTracker != null) {
+              tracker = oldTracker;
+            }
+            recoverTrackerResources(tracker, trackerState);
+          }
+        }
       }
     }
   }
@@ -556,7 +562,7 @@ public class ResourceLocalizationService extends CompositeService
       rsrcCleanup.getResources();
     for (Map.Entry<LocalResourceVisibility, Collection<LocalResourceRequest>> e :
          rsrcs.entrySet()) {
-      LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(), 
+      LocalResourcesTracker tracker = getLocalResourcesTracker(e.getKey(), c.getUser(),
           c.getContainerId().getApplicationAttemptId()
           .getApplicationId());
       for (LocalResourceRequest req : e.getValue()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index 67f642d..5d4253d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -73,6 +74,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -225,68 +227,119 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     return isHealthy;
   }
 
-  @Override
-  public List<RecoveredContainerState> loadContainersState()
+  // LeveldbIterator starting at startkey
+  private LeveldbIterator getLevelDBIterator(String startKey)
       throws IOException {
-    ArrayList<RecoveredContainerState> containers =
-        new ArrayList<RecoveredContainerState>();
-    ArrayList<ContainerId> containersToRemove =
-              new ArrayList<ContainerId>();
-    LeveldbIterator iter = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(CONTAINERS_KEY_PREFIX));
+      LeveldbIterator it = new LeveldbIterator(db);
+      it.seek(bytes(startKey));
+      return it;
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
 
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.peekNext();
+  // Base Recovery Iterator
+  private abstract class BaseRecoveryIterator<T> implements
+      RecoveryIterator<T> {
+    LeveldbIterator it;
+    T nextItem;
+
+    BaseRecoveryIterator(String dbKey) throws IOException {
+      this.it = getLevelDBIterator(dbKey);
+      this.nextItem = null;
+    }
+
+    protected abstract T getNextItem(LeveldbIterator it) throws IOException;
+
+    @Override
+    public boolean hasNext() throws IOException {
+      if (nextItem == null) {
+        nextItem = getNextItem(it);
+      }
+      return (nextItem != null);
+    }
+
+    @Override
+    public T next() throws IOException, NoSuchElementException {
+      T tmp = nextItem;
+      if (tmp != null) {
+        nextItem = null;
+        return tmp;
+      } else {
+        tmp = getNextItem(it);
+        if (tmp == null) {
+          throw new NoSuchElementException();
+        }
+        return tmp;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (it != null) {
+        it.close();
+      }
+    }
+  }
+
+  //  Container Recovery Iterator
+  private class ContainerStateIterator extends
+      BaseRecoveryIterator<RecoveredContainerState> {
+    ContainerStateIterator() throws IOException {
+      super(CONTAINERS_KEY_PREFIX);
+    }
+
+    @Override
+    protected RecoveredContainerState getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredContainer(it);
+    }
+  }
+
+  private RecoveredContainerState getNextRecoveredContainer(LeveldbIterator it)
+      throws IOException {
+    RecoveredContainerState rcs = null;
+    try {
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.peekNext();
         String key = asString(entry.getKey());
         if (!key.startsWith(CONTAINERS_KEY_PREFIX)) {
-          break;
+          return null;
         }
 
         int idEndPos = key.indexOf('/', CONTAINERS_KEY_PREFIX.length());
         if (idEndPos < 0) {
           throw new IOException("Unable to determine container in key: " + key);
         }
-        ContainerId containerId = ContainerId.fromString(
-            key.substring(CONTAINERS_KEY_PREFIX.length(), idEndPos));
-        String keyPrefix = key.substring(0, idEndPos+1);
-        RecoveredContainerState rcs = loadContainerState(containerId,
-            iter, keyPrefix);
-        // Don't load container without StartContainerRequest
+        String keyPrefix = key.substring(0, idEndPos + 1);
+        rcs = loadContainerState(it, keyPrefix);
         if (rcs.startRequest != null) {
-          containers.add(rcs);
+          break;
         } else {
-          containersToRemove.add(containerId);
+          removeContainer(rcs.getContainerId());
+          rcs = null;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return rcs;
+  }
 
-    // remove container without StartContainerRequest
-    for (ContainerId containerId : containersToRemove) {
-      LOG.warn("Remove container " + containerId +
-          " with incomplete records");
-      try {
-        removeContainer(containerId);
-        // TODO: kill and cleanup the leaked container
-      } catch (IOException e) {
-        LOG.error("Unable to remove container " + containerId +
-            " in store", e);
-      }
-    }
 
-    return containers;
+  @Override
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
+      throws IOException {
+    return new ContainerStateIterator();
   }
 
-  private RecoveredContainerState loadContainerState(ContainerId containerId,
-      LeveldbIterator iter, String keyPrefix) throws IOException {
-    RecoveredContainerState rcs = new RecoveredContainerState();
+  private RecoveredContainerState loadContainerState(LeveldbIterator iter,
+       String keyPrefix) throws IOException {
+    ContainerId containerId = ContainerId.fromString(
+        keyPrefix.substring(CONTAINERS_KEY_PREFIX.length(),
+            keyPrefix.length()-1));
+    RecoveredContainerState rcs = new RecoveredContainerState(containerId);
     rcs.status = RecoveredContainerStatus.REQUESTED;
     while (iter.hasNext()) {
       Entry<byte[],byte[]> entry = iter.peekNext();
@@ -680,35 +733,45 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
 
-  @Override
-  public RecoveredApplicationsState loadApplicationsState()
-      throws IOException {
-    RecoveredApplicationsState state = new RecoveredApplicationsState();
-    state.applications = new ArrayList<ContainerManagerApplicationProto>();
-    String keyPrefix = APPLICATIONS_KEY_PREFIX;
-    LeveldbIterator iter = null;
+  // Application Recovery Iterator
+  private class ApplicationStateIterator extends
+      BaseRecoveryIterator<ContainerManagerApplicationProto> {
+    ApplicationStateIterator() throws IOException {
+      super(APPLICATIONS_KEY_PREFIX);
+    }
+
+    @Override
+    protected ContainerManagerApplicationProto getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredApplication(it);
+    }
+  }
+
+  private ContainerManagerApplicationProto getNextRecoveredApplication(
+      LeveldbIterator it) throws IOException {
+    ContainerManagerApplicationProto applicationProto = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(keyPrefix));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String key = asString(entry.getKey());
-        if (!key.startsWith(keyPrefix)) {
-          break;
+        if (!key.startsWith(APPLICATIONS_KEY_PREFIX)) {
+          return null;
         }
-        state.applications.add(
-            ContainerManagerApplicationProto.parseFrom(entry.getValue()));
+        applicationProto = ContainerManagerApplicationProto.parseFrom(
+            entry.getValue());
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return applicationProto;
+  }
 
+  @Override
+  public RecoveredApplicationsState loadApplicationsState()
+      throws IOException {
+    RecoveredApplicationsState state = new RecoveredApplicationsState();
+    state.it = new ApplicationStateIterator();
     cleanupDeprecatedFinishedApps();
-
     return state;
   }
 
@@ -752,24 +815,29 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
 
-  @Override
-  public RecoveredLocalizationState loadLocalizationState()
-      throws IOException {
-    RecoveredLocalizationState state = new RecoveredLocalizationState();
+  // User Resource Recovery Iterator.
+  private class UserResourcesIterator extends
+      BaseRecoveryIterator<Entry<String, RecoveredUserResources>> {
+    UserResourcesIterator() throws IOException {
+      super(LOCALIZATION_PRIVATE_KEY_PREFIX);
+    }
 
-    LeveldbIterator iter = null;
-    try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(LOCALIZATION_PUBLIC_KEY_PREFIX));
-      state.publicTrackerState = loadResourceTrackerState(iter,
-          LOCALIZATION_PUBLIC_KEY_PREFIX);
+    @Override
+    protected Entry<String, RecoveredUserResources> getNextItem(
+        LeveldbIterator it) throws IOException {
+      return getNextRecoveredPrivateLocalizationEntry(it);
+    }
+  }
 
-      iter.seek(bytes(LOCALIZATION_PRIVATE_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[],byte[]> entry = iter.peekNext();
+  private Entry<String, RecoveredUserResources> getNextRecoveredPrivateLocalizationEntry(
+      LeveldbIterator it) throws IOException {
+    Entry<String, RecoveredUserResources> localEntry = null;
+    try {
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.peekNext();
         String key = asString(entry.getKey());
         if (!key.startsWith(LOCALIZATION_PRIVATE_KEY_PREFIX)) {
-          break;
+          return null;
         }
 
         int userEndPos = key.indexOf('/',
@@ -780,17 +848,24 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         }
         String user = key.substring(
             LOCALIZATION_PRIVATE_KEY_PREFIX.length(), userEndPos);
-        state.userResources.put(user, loadUserLocalizedResources(iter,
-            key.substring(0, userEndPos+1)));
+        RecoveredUserResources val = loadUserLocalizedResources(it,
+            key.substring(0, userEndPos+1));
+        localEntry = new AbstractMap.SimpleEntry<>(user, val);
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return localEntry;
+  }
 
+  @Override
+  public RecoveredLocalizationState loadLocalizationState()
+      throws IOException {
+    RecoveredLocalizationState state = new RecoveredLocalizationState();
+    LeveldbIterator it = getLevelDBIterator(LOCALIZATION_PUBLIC_KEY_PREFIX);
+    state.publicTrackerState = loadResourceTrackerState(it,
+        LOCALIZATION_PUBLIC_KEY_PREFIX);
+    state.it = new UserResourcesIterator();
     return state;
   }
 
@@ -800,7 +875,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     final String startedPrefix = keyPrefix + LOCALIZATION_STARTED_SUFFIX;
     LocalResourceTrackerState state = new LocalResourceTrackerState();
     while (iter.hasNext()) {
-      Entry<byte[],byte[]> entry = iter.peekNext();
+      Entry<byte[], byte[]> entry = iter.peekNext();
       String key = asString(entry.getKey());
       if (!key.startsWith(keyPrefix)) {
         break;
@@ -981,32 +1056,44 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         + LOCALIZATION_APPCACHE_SUFFIX + appId + "/";
   }
 
+  // Deletion State Recovery Iterator.
+  private class DeletionStateIterator extends
+      BaseRecoveryIterator<DeletionServiceDeleteTaskProto> {
+    DeletionStateIterator() throws IOException {
+      super(DELETION_TASK_KEY_PREFIX);
+    }
 
-  @Override
-  public RecoveredDeletionServiceState loadDeletionServiceState()
-      throws IOException {
-    RecoveredDeletionServiceState state = new RecoveredDeletionServiceState();
-    state.tasks = new ArrayList<DeletionServiceDeleteTaskProto>();
-    LeveldbIterator iter = null;
+    @Override
+    protected DeletionServiceDeleteTaskProto getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextRecoveredDeletionService(it);
+    }
+  }
+
+  private DeletionServiceDeleteTaskProto getNextRecoveredDeletionService(
+      LeveldbIterator it) throws IOException {
+    DeletionServiceDeleteTaskProto deleteProto = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(DELETION_TASK_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      if (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String key = asString(entry.getKey());
         if (!key.startsWith(DELETION_TASK_KEY_PREFIX)) {
-          break;
+          return null;
         }
-        state.tasks.add(
-            DeletionServiceDeleteTaskProto.parseFrom(entry.getValue()));
+        deleteProto = DeletionServiceDeleteTaskProto.parseFrom(
+            entry.getValue());
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return deleteProto;
+  }
+
+  @Override
+  public RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException {
+    RecoveredDeletionServiceState state = new RecoveredDeletionServiceState();
+    state.it = new DeletionStateIterator();
     return state;
   }
 
@@ -1033,29 +1120,44 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     }
   }
 
+  private MasterKey getMasterKey(String dbKey) throws IOException {
+    try{
+      byte[] data = db.get(bytes(dbKey));
+      if (data == null || data.length == 0) {
+        return null;
+      }
+      return parseMasterKey(data);
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
 
-  @Override
-  public RecoveredNMTokensState loadNMTokensState() throws IOException {
-    RecoveredNMTokensState state = new RecoveredNMTokensState();
-    state.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>();
-    LeveldbIterator iter = null;
+  // Recover NMTokens Iterator
+  private class NMTokensStateIterator extends
+      BaseRecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> {
+    NMTokensStateIterator() throws IOException {
+      super(NM_TOKENS_KEY_PREFIX);
+    }
+
+    @Override
+    protected Entry<ApplicationAttemptId, MasterKey> getNextItem(
+        LeveldbIterator it) throws IOException {
+      return getNextMasterKeyEntry(it);
+    }
+  }
+
+  private Entry<ApplicationAttemptId, MasterKey> getNextMasterKeyEntry(
+      LeveldbIterator it) throws IOException {
+    Entry<ApplicationAttemptId, MasterKey> masterKeyentry = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(NM_TOKENS_KEY_PREFIX));
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String fullKey = asString(entry.getKey());
         if (!fullKey.startsWith(NM_TOKENS_KEY_PREFIX)) {
           break;
         }
         String key = fullKey.substring(NM_TOKENS_KEY_PREFIX.length());
-        if (key.equals(CURRENT_MASTER_KEY_SUFFIX)) {
-          state.currentMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.equals(PREV_MASTER_KEY_SUFFIX)) {
-          state.previousMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.startsWith(
-            ApplicationAttemptId.appAttemptIdStrPrefix)) {
+        if (key.startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
           ApplicationAttemptId attempt;
           try {
             attempt = ApplicationAttemptId.fromString(key);
@@ -1063,17 +1165,25 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
             throw new IOException("Bad application master key state for "
                 + fullKey, e);
           }
-          state.applicationMasterKeys.put(attempt,
+          masterKeyentry = new AbstractMap.SimpleEntry<>(attempt,
               parseMasterKey(entry.getValue()));
+          break;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
+    return masterKeyentry;
+  }
+
+  @Override
+  public RecoveredNMTokensState loadNMTokensState() throws IOException {
+    RecoveredNMTokensState state = new RecoveredNMTokensState();
+    state.currentMasterKey = getMasterKey(NM_TOKENS_KEY_PREFIX
+                                          + CURRENT_MASTER_KEY_SUFFIX);
+    state.previousMasterKey = getMasterKey(NM_TOKENS_KEY_PREFIX
+                                            + PREV_MASTER_KEY_SUFFIX);
+    state.it = new NMTokensStateIterator();
     return state;
   }
 
@@ -1122,45 +1232,45 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     }
   }
 
+  // Recover ContainersToken Iterator.
+  private class ContainerTokensStateIterator extends
+      BaseRecoveryIterator<Entry<ContainerId, Long>> {
+    ContainerTokensStateIterator() throws IOException {
+      super(CONTAINER_TOKENS_KEY_PREFIX);
+    }
 
-  @Override
-  public RecoveredContainerTokensState loadContainerTokensState()
+    @Override
+    protected Entry<ContainerId, Long> getNextItem(LeveldbIterator it)
+        throws IOException {
+      return getNextContainerToken(it);
+    }
+  }
+
+  private Entry<ContainerId, Long> getNextContainerToken(LeveldbIterator it)
       throws IOException {
-    RecoveredContainerTokensState state = new RecoveredContainerTokensState();
-    state.activeTokens = new HashMap<ContainerId, Long>();
-    LeveldbIterator iter = null;
+    Entry<ContainerId, Long> containerTokenEntry = null;
     try {
-      iter = new LeveldbIterator(db);
-      iter.seek(bytes(CONTAINER_TOKENS_KEY_PREFIX));
-      final int containerTokensKeyPrefixLength =
-          CONTAINER_TOKENS_KEY_PREFIX.length();
-      while (iter.hasNext()) {
-        Entry<byte[], byte[]> entry = iter.next();
+      while (it.hasNext()) {
+        Entry<byte[], byte[]> entry = it.next();
         String fullKey = asString(entry.getKey());
         if (!fullKey.startsWith(CONTAINER_TOKENS_KEY_PREFIX)) {
           break;
         }
-        String key = fullKey.substring(containerTokensKeyPrefixLength);
-        if (key.equals(CURRENT_MASTER_KEY_SUFFIX)) {
-          state.currentMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.equals(PREV_MASTER_KEY_SUFFIX)) {
-          state.previousMasterKey = parseMasterKey(entry.getValue());
-        } else if (key.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
-          loadContainerToken(state, fullKey, key, entry.getValue());
+        String key = fullKey.substring(CONTAINER_TOKENS_KEY_PREFIX.length());
+        if (key.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
+          containerTokenEntry = loadContainerToken(fullKey, key,
+              entry.getValue());
+          break;
         }
       }
     } catch (DBException e) {
       throw new IOException(e);
-    } finally {
-      if (iter != null) {
-        iter.close();
-      }
     }
-    return state;
+    return containerTokenEntry;
   }
 
-  private static void loadContainerToken(RecoveredContainerTokensState state,
-      String key, String containerIdStr, byte[] value) throws IOException {
+  private static Entry<ContainerId, Long> loadContainerToken(String key,
+      String containerIdStr, byte[] value) throws IOException {
     ContainerId containerId;
     Long expTime;
     try {
@@ -1169,7 +1279,19 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     } catch (IllegalArgumentException e) {
       throw new IOException("Bad container token state for " + key, e);
     }
-    state.activeTokens.put(containerId, expTime);
+    return new AbstractMap.SimpleEntry<>(containerId, expTime);
+  }
+
+  @Override
+  public RecoveredContainerTokensState loadContainerTokensState()
+      throws IOException {
+    RecoveredContainerTokensState state = new RecoveredContainerTokensState();
+    state.currentMasterKey = getMasterKey(CONTAINER_TOKENS_KEY_PREFIX
+        + CURRENT_MASTER_KEY_SUFFIX);
+    state.previousMasterKey = getMasterKey(CONTAINER_TOKENS_KEY_PREFIX
+        + PREV_MASTER_KEY_SUFFIX);
+    state.it = new ContainerTokensStateIterator();
+    return state;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index dfad9cf..3ae00f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -65,7 +65,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public List<RecoveredContainerState> loadContainersState()
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException {
     throw new UnsupportedOperationException(
         "Recovery not supported by this state store");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
index 70decdb..35caec9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -67,12 +68,11 @@ public abstract class NMStateStoreService extends AbstractService {
   }
 
   public static class RecoveredApplicationsState {
-    List<ContainerManagerApplicationProto> applications;
+    RecoveryIterator<ContainerManagerApplicationProto> it = null;
 
-    public List<ContainerManagerApplicationProto> getApplications() {
-      return applications;
+    public RecoveryIterator<ContainerManagerApplicationProto> getIterator() {
+      return it;
     }
-
   }
 
   /**
@@ -106,6 +106,15 @@ public abstract class NMStateStoreService extends AbstractService {
         RecoveredContainerType.RECOVER;
     private long startTime;
     private ResourceMappings resMappings = new ResourceMappings();
+    private final ContainerId containerId;
+
+    RecoveredContainerState(ContainerId containerId){
+      this.containerId = containerId;
+    }
+
+    public ContainerId getContainerId() {
+      return containerId;
+    }
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -248,30 +257,33 @@ public abstract class NMStateStoreService extends AbstractService {
   public static class RecoveredLocalizationState {
     LocalResourceTrackerState publicTrackerState =
         new LocalResourceTrackerState();
-    Map<String, RecoveredUserResources> userResources =
-        new HashMap<String, RecoveredUserResources>();
+    RecoveryIterator<Entry<String, RecoveredUserResources>> it = null;
 
     public LocalResourceTrackerState getPublicTrackerState() {
       return publicTrackerState;
     }
 
-    public Map<String, RecoveredUserResources> getUserResources() {
-      return userResources;
+    public RecoveryIterator<Entry<String, RecoveredUserResources>> getIterator() {
+      return it;
     }
   }
 
   public static class RecoveredDeletionServiceState {
-    List<DeletionServiceDeleteTaskProto> tasks;
+    RecoveryIterator<DeletionServiceDeleteTaskProto> it = null;
 
-    public List<DeletionServiceDeleteTaskProto> getTasks() {
-      return tasks;
+    public RecoveryIterator<DeletionServiceDeleteTaskProto> getIterator(){
+      return it;
     }
   }
 
   public static class RecoveredNMTokensState {
     MasterKey currentMasterKey;
     MasterKey previousMasterKey;
-    Map<ApplicationAttemptId, MasterKey> applicationMasterKeys;
+    RecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> it = null;
+
+    public RecoveryIterator<Entry<ApplicationAttemptId, MasterKey>> getIterator() {
+      return it;
+    }
 
     public MasterKey getCurrentMasterKey() {
       return currentMasterKey;
@@ -281,15 +293,16 @@ public abstract class NMStateStoreService extends AbstractService {
       return previousMasterKey;
     }
 
-    public Map<ApplicationAttemptId, MasterKey> getApplicationMasterKeys() {
-      return applicationMasterKeys;
-    }
   }
 
   public static class RecoveredContainerTokensState {
     MasterKey currentMasterKey;
     MasterKey previousMasterKey;
-    Map<ContainerId, Long> activeTokens;
+    RecoveryIterator<Entry<ContainerId, Long>> it = null;
+
+    public RecoveryIterator<Entry<ContainerId, Long>> getIterator() {
+      return it;
+    }
 
     public MasterKey getCurrentMasterKey() {
       return currentMasterKey;
@@ -299,9 +312,6 @@ public abstract class NMStateStoreService extends AbstractService {
       return previousMasterKey;
     }
 
-    public Map<ContainerId, Long> getActiveTokens() {
-      return activeTokens;
-    }
   }
 
   public static class RecoveredLogDeleterState {
@@ -400,11 +410,10 @@ public abstract class NMStateStoreService extends AbstractService {
 
 
   /**
-   * Load the state of containers
-   * @return recovered state for containers
-   * @throws IOException
+   * get the Recovered Container State Iterator
+   * @return recovery iterator
    */
-  public abstract List<RecoveredContainerState> loadContainersState()
+  public abstract RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java
new file mode 100644
index 0000000..0bb262a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/RecoveryIterator.java
@@ -0,0 +1,41 @@
+/**
+ * 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.server.nodemanager.recovery;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+/**
+ * A wrapper for a Iterator to translate the raw RuntimeExceptions that
+ * can be thrown into IOException.
+ */
+public interface RecoveryIterator<T> extends Closeable {
+
+  /**
+   * Returns true if the iteration has more elements.
+   */
+  boolean hasNext() throws IOException;
+
+  /**
+   * Returns the next element in the iteration.
+   */
+  T next() throws IOException, NoSuchElementException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
index 256f649..b3df69b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMContainerTokenSecretManager.java
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.TreeMap;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -90,17 +92,20 @@ public class NMContainerTokenSecretManager extends
       super.serialNo = super.currentMasterKey.getMasterKey().getKeyId() + 1;
     }
 
-    for (Entry<ContainerId, Long> entry : state.getActiveTokens().entrySet()) {
-      ContainerId containerId = entry.getKey();
-      Long expTime = entry.getValue();
-      List<ContainerId> containerList =
-          recentlyStartedContainerTracker.get(expTime);
-      if (containerList == null) {
-        containerList = new ArrayList<ContainerId>();
-        recentlyStartedContainerTracker.put(expTime, containerList);
-      }
-      if (!containerList.contains(containerId)) {
-        containerList.add(containerId);
+    try (RecoveryIterator<Entry<ContainerId, Long>> it = state.getIterator()) {
+      while (it.hasNext()) {
+        Entry<ContainerId, Long> entry = it.next();
+        ContainerId containerId = entry.getKey();
+        Long expTime = entry.getValue();
+        List<ContainerId> containerList =
+            recentlyStartedContainerTracker.get(expTime);
+        if (containerList == null) {
+          containerList = new ArrayList<ContainerId>();
+          recentlyStartedContainerTracker.put(expTime, containerList);
+        }
+        if (!containerList.contains(containerId)) {
+          containerList.add(containerId);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
index 0956e77..f895791 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
+import org.apache.hadoop.yarn.server.nodemanager.recovery.RecoveryIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,11 +89,14 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
       super.serialNo = super.currentMasterKey.getMasterKey().getKeyId() + 1;
     }
 
-    for (Map.Entry<ApplicationAttemptId, MasterKey> entry :
-         state.getApplicationMasterKeys().entrySet()) {
-      key = entry.getValue();
-      oldMasterKeys.put(entry.getKey(),
-          new MasterKeyData(key, createSecretKey(key.getBytes().array())));
+    try (RecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>> it =
+             state.getIterator()) {
+      while (it.hasNext()) {
+        Map.Entry<ApplicationAttemptId, MasterKey> entry = it.next();
+        key = entry.getValue();
+        oldMasterKeys.put(entry.getKey(),
+            new MasterKeyData(key, createSecretKey(key.getBytes().array())));
+      }
     }
 
     // reconstruct app to app attempts map

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
index c5428d1..9658ecd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
@@ -23,6 +23,7 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -56,6 +57,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   private Map<Integer, DeletionServiceDeleteTaskProto> deleteTasks;
   private RecoveredNMTokensState nmTokenState;
   private RecoveredContainerTokensState containerTokenState;
+  private Map<ApplicationAttemptId, MasterKey> applicationMasterKeys;
+  private Map<ContainerId, Long> activeTokens;
   private Map<ApplicationId, LogDeleterProto> logDeleterState;
   private RecoveredAMRMProxyState amrmProxyState;
 
@@ -68,10 +71,9 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
     apps = new HashMap<ApplicationId, ContainerManagerApplicationProto>();
     containerStates = new HashMap<ContainerId, RecoveredContainerState>();
     nmTokenState = new RecoveredNMTokensState();
-    nmTokenState.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>();
+    applicationMasterKeys = new HashMap<ApplicationAttemptId, MasterKey>();
     containerTokenState = new RecoveredContainerTokensState();
-    containerTokenState.activeTokens = new HashMap<ContainerId, Long>();
+    activeTokens = new HashMap<ContainerId, Long>();
     trackerStates = new HashMap<TrackerKey, TrackerState>();
     deleteTasks = new HashMap<Integer, DeletionServiceDeleteTaskProto>();
     logDeleterState = new HashMap<ApplicationId, LogDeleterProto>();
@@ -86,13 +88,39 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   protected void closeStorage() {
   }
 
+  // Recovery Iterator Implementation.
+  private class NMMemoryRecoveryIterator<T> implements RecoveryIterator<T> {
+
+    private Iterator<T> it;
+
+    NMMemoryRecoveryIterator(Iterator<T> it){
+      this.it = it;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return it.hasNext();
+    }
+
+    @Override
+    public T next() throws IOException {
+      return it.next();
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+  }
 
   @Override
   public synchronized RecoveredApplicationsState loadApplicationsState()
       throws IOException {
     RecoveredApplicationsState state = new RecoveredApplicationsState();
-    state.applications = new ArrayList<ContainerManagerApplicationProto>(
-        apps.values());
+    List<ContainerManagerApplicationProto> containerList =
+        new ArrayList<ContainerManagerApplicationProto>(apps.values());
+    state.it = new NMMemoryRecoveryIterator<ContainerManagerApplicationProto>(
+        containerList.iterator());
     return state;
   }
 
@@ -111,13 +139,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public synchronized List<RecoveredContainerState> loadContainersState()
+  public RecoveryIterator<RecoveredContainerState> getContainerStateIterator()
       throws IOException {
     // return a copy so caller can't modify our state
     List<RecoveredContainerState> result =
         new ArrayList<RecoveredContainerState>(containerStates.size());
     for (RecoveredContainerState rcs : containerStates.values()) {
-      RecoveredContainerState rcsCopy = new RecoveredContainerState();
+      RecoveredContainerState rcsCopy = new RecoveredContainerState(rcs.getContainerId());
       rcsCopy.status = rcs.status;
       rcsCopy.exitCode = rcs.exitCode;
       rcsCopy.killed = rcs.killed;
@@ -131,13 +159,14 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       rcsCopy.setResourceMappings(rcs.getResourceMappings());
       result.add(rcsCopy);
     }
-    return result;
+    return new NMMemoryRecoveryIterator<RecoveredContainerState>(
+        result.iterator());
   }
 
   @Override
   public synchronized void storeContainer(ContainerId containerId,
       int version, long startTime, StartContainerRequest startRequest) {
-    RecoveredContainerState rcs = new RecoveredContainerState();
+    RecoveredContainerState rcs = new RecoveredContainerState(containerId);
     rcs.startRequest = startRequest;
     rcs.version = version;
     try {
@@ -284,6 +313,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   @Override
   public synchronized RecoveredLocalizationState loadLocalizationState() {
     RecoveredLocalizationState result = new RecoveredLocalizationState();
+    Map<String, RecoveredUserResources> userResources =
+        new HashMap<String, RecoveredUserResources>();
     for (Map.Entry<TrackerKey, TrackerState> e : trackerStates.entrySet()) {
       TrackerKey tk = e.getKey();
       TrackerState ts = e.getValue();
@@ -294,10 +325,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       if (tk.user == null) {
         result.publicTrackerState = loadTrackerState(ts);
       } else {
-        RecoveredUserResources rur = result.userResources.get(tk.user);
+        RecoveredUserResources rur = userResources.get(tk.user);
         if (rur == null) {
           rur = new RecoveredUserResources();
-          result.userResources.put(tk.user, rur);
+          userResources.put(tk.user, rur);
         }
         if (tk.appId == null) {
           rur.privateTrackerState = loadTrackerState(ts);
@@ -306,6 +337,8 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
         }
       }
     }
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<String, RecoveredUserResources>>(
+        userResources.entrySet().iterator());
     return result;
   }
 
@@ -341,8 +374,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       throws IOException {
     RecoveredDeletionServiceState result =
         new RecoveredDeletionServiceState();
-    result.tasks = new ArrayList<DeletionServiceDeleteTaskProto>(
-        deleteTasks.values());
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        new ArrayList<DeletionServiceDeleteTaskProto>(deleteTasks.values());
+    result.it = new NMMemoryRecoveryIterator<DeletionServiceDeleteTaskProto>(
+        deleteTaskProtos.iterator());
     return result;
   }
 
@@ -365,9 +400,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
     RecoveredNMTokensState result = new RecoveredNMTokensState();
     result.currentMasterKey = nmTokenState.currentMasterKey;
     result.previousMasterKey = nmTokenState.previousMasterKey;
-    result.applicationMasterKeys =
-        new HashMap<ApplicationAttemptId, MasterKey>(
-            nmTokenState.applicationMasterKeys);
+    Map<ApplicationAttemptId, MasterKey> masterKeysMap =
+        new HashMap<ApplicationAttemptId, MasterKey>(applicationMasterKeys);
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>>(
+        masterKeysMap.entrySet().iterator());
     return result;
   }
 
@@ -389,14 +425,14 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   public synchronized void storeNMTokenApplicationMasterKey(
       ApplicationAttemptId attempt, MasterKey key) throws IOException {
     MasterKeyPBImpl keypb = (MasterKeyPBImpl) key;
-    nmTokenState.applicationMasterKeys.put(attempt,
+    applicationMasterKeys.put(attempt,
         new MasterKeyPBImpl(keypb.getProto()));
   }
 
   @Override
   public synchronized void removeNMTokenApplicationMasterKey(
       ApplicationAttemptId attempt) throws IOException {
-    nmTokenState.applicationMasterKeys.remove(attempt);
+    applicationMasterKeys.remove(attempt);
   }
 
 
@@ -408,8 +444,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
         new RecoveredContainerTokensState();
     result.currentMasterKey = containerTokenState.currentMasterKey;
     result.previousMasterKey = containerTokenState.previousMasterKey;
-    result.activeTokens =
-        new HashMap<ContainerId, Long>(containerTokenState.activeTokens);
+    Map<ContainerId, Long> containersTokenMap =
+        new HashMap<ContainerId, Long>(activeTokens);
+    result.it = new NMMemoryRecoveryIterator<Map.Entry<ContainerId, Long>>(
+        containersTokenMap.entrySet().iterator());
     return result;
   }
 
@@ -432,13 +470,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   @Override
   public synchronized void storeContainerToken(ContainerId containerId,
       Long expirationTime) throws IOException {
-    containerTokenState.activeTokens.put(containerId, expirationTime);
+    activeTokens.put(containerId, expirationTime);
   }
 
   @Override
   public synchronized void removeContainerToken(ContainerId containerId)
       throws IOException {
-    containerTokenState.activeTokens.remove(containerId);
+    activeTokens.remove(containerId);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65e74697/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index 8a8cfa2..fcbbc52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -125,6 +125,73 @@ public class TestNMLeveldbStateStoreService {
     FileUtil.fullyDelete(TMP_DIR);
   }
 
+  private List<RecoveredContainerState> loadContainersState(
+      RecoveryIterator<RecoveredContainerState> it) throws IOException {
+    List<RecoveredContainerState> containers =
+        new ArrayList<RecoveredContainerState>();
+    while (it.hasNext()) {
+      RecoveredContainerState rcs = it.next();
+      containers.add(rcs);
+    }
+    return containers;
+  }
+
+  private List<ContainerManagerApplicationProto> loadApplicationProtos(
+      RecoveryIterator<ContainerManagerApplicationProto> it)
+      throws IOException {
+    List<ContainerManagerApplicationProto> applicationProtos =
+        new ArrayList<ContainerManagerApplicationProto>();
+    while (it.hasNext()) {
+      applicationProtos.add(it.next());
+    }
+    return applicationProtos;
+  }
+
+  private List<DeletionServiceDeleteTaskProto> loadDeletionTaskProtos(
+      RecoveryIterator<DeletionServiceDeleteTaskProto> it) throws IOException {
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        new ArrayList<DeletionServiceDeleteTaskProto>();
+    while (it.hasNext()) {
+      deleteTaskProtos.add(it.next());
+    }
+    return deleteTaskProtos;
+  }
+
+  private Map<String, RecoveredUserResources> loadUserResources(
+      RecoveryIterator<Map.Entry<String, RecoveredUserResources>> it)
+      throws IOException {
+    Map<String, RecoveredUserResources> userResources =
+        new HashMap<String, RecoveredUserResources>();
+    while (it.hasNext()) {
+      Map.Entry<String, RecoveredUserResources> entry = it.next();
+      userResources.put(entry.getKey(), entry.getValue());
+    }
+    return userResources;
+  }
+
+  private Map<ApplicationAttemptId, MasterKey> loadNMTokens(
+      RecoveryIterator<Map.Entry<ApplicationAttemptId, MasterKey>> it)
+      throws IOException {
+    Map<ApplicationAttemptId, MasterKey> nmTokens =
+        new HashMap<ApplicationAttemptId, MasterKey>();
+    while (it.hasNext()) {
+      Map.Entry<ApplicationAttemptId, MasterKey> entry = it.next();
+      nmTokens.put(entry.getKey(), entry.getValue());
+    }
+    return nmTokens;
+  }
+
+  private Map<ContainerId, Long> loadContainerTokens(
+      RecoveryIterator<Map.Entry<ContainerId, Long>> it) throws IOException {
+    Map<ContainerId, Long> containerTokens =
+        new HashMap<ContainerId, Long>();
+    while (it.hasNext()) {
+      Map.Entry<ContainerId, Long> entry = it.next();
+      containerTokens.put(entry.getKey(), entry.getValue());
+    }
+    return containerTokens;
+  }
+
   private void restartStateStore() throws IOException {
     // need to close so leveldb releases database lock
     if (stateStore != null) {
@@ -142,7 +209,7 @@ public class TestNMLeveldbStateStoreService {
     assertNotNull(pubts);
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
-    assertTrue(state.getUserResources().isEmpty());
+    assertTrue(loadUserResources(state.getIterator()).isEmpty());
   }
 
   @Test
@@ -183,7 +250,7 @@ public class TestNMLeveldbStateStoreService {
       restartStateStore();
       Assert.fail("Incompatible version, should expect fail here.");
     } catch (ServiceStateException e) {
-      Assert.assertTrue("Exception message mismatch", 
+      Assert.assertTrue("Exception message mismatch",
         e.getMessage().contains("Incompatible version for NM state:"));
     }
   }
@@ -192,7 +259,9 @@ public class TestNMLeveldbStateStoreService {
   public void testApplicationStorage() throws IOException {
     // test empty when no state
     RecoveredApplicationsState state = stateStore.loadApplicationsState();
-    assertTrue(state.getApplications().isEmpty());
+    List<ContainerManagerApplicationProto> apps =
+        loadApplicationProtos(state.getIterator());
+    assertTrue(apps.isEmpty());
 
     // store an application and verify recovered
     final ApplicationId appId1 = ApplicationId.newInstance(1234, 1);
@@ -204,8 +273,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeApplication(appId1, appProto1);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(1, state.getApplications().size());
-    assertEquals(appProto1, state.getApplications().get(0));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(1, apps.size());
+    assertEquals(appProto1, apps.get(0));
 
     // add a new app
     final ApplicationId appId2 = ApplicationId.newInstance(1234, 2);
@@ -216,23 +286,25 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeApplication(appId2, appProto2);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(2, state.getApplications().size());
-    assertTrue(state.getApplications().contains(appProto1));
-    assertTrue(state.getApplications().contains(appProto2));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(2, apps.size());
+    assertTrue(apps.contains(appProto1));
+    assertTrue(apps.contains(appProto2));
 
     // test removing an application
     stateStore.removeApplication(appId2);
     restartStateStore();
     state = stateStore.loadApplicationsState();
-    assertEquals(1, state.getApplications().size());
-    assertEquals(appProto1, state.getApplications().get(0));
+    apps = loadApplicationProtos(state.getIterator());
+    assertEquals(1, apps.size());
+    assertEquals(appProto1, apps.get(0));
   }
 
   @Test
   public void testContainerStorage() throws IOException {
     // test empty when no state
     List<RecoveredContainerState> recoveredContainers =
-        stateStore.loadContainersState();
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     // create a container request
@@ -254,7 +326,8 @@ public class TestNMLeveldbStateStoreService {
         stateStore.getContainerVersionKey(containerId.toString()))));
 
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     assertEquals(0, rcs.getVersion());
@@ -269,14 +342,16 @@ public class TestNMLeveldbStateStoreService {
     // store a new container record without StartContainerRequest
     ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 6);
     stateStore.storeContainerLaunched(containerId1);
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     // check whether the new container record is discarded
     assertEquals(1, recoveredContainers.size());
 
     // queue the container, and verify recovered
     stateStore.storeContainerQueued(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.QUEUED, rcs.getStatus());
@@ -292,7 +367,8 @@ public class TestNMLeveldbStateStoreService {
     diags.append("some diags for container");
     stateStore.storeContainerDiagnostics(containerId, diags);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
@@ -305,7 +381,8 @@ public class TestNMLeveldbStateStoreService {
     // pause the container, and verify recovered
     stateStore.storeContainerPaused(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.PAUSED, rcs.getStatus());
@@ -316,7 +393,8 @@ public class TestNMLeveldbStateStoreService {
     // Resume the container
     stateStore.removeContainerPaused(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
 
     // increase the container size, and verify recovered
@@ -328,7 +406,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore
         .storeContainerUpdateToken(containerId, updateTokenIdentifier);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(0, rcs.getVersion());
@@ -342,7 +421,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerDiagnostics(containerId, diags);
     stateStore.storeContainerKilled(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
@@ -358,7 +438,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerDiagnostics(containerId, diags);
     stateStore.storeContainerCompleted(containerId, 21);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.COMPLETED, rcs.getStatus());
@@ -371,7 +452,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerWorkDir(containerId, "/test/workdir");
     stateStore.storeContainerLogDir(containerId, "/test/logdir");
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
     assertEquals(6, rcs.getRemainingRetryAttempts());
@@ -382,12 +464,13 @@ public class TestNMLeveldbStateStoreService {
     // remove the container and verify not recovered
     stateStore.removeContainer(containerId);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
     // recover again to check remove clears all containers
     restartStateStore();
     NMStateStoreService nmStoreSpy = spy(stateStore);
-    nmStoreSpy.loadContainersState();
+    loadContainersState(nmStoreSpy.getContainerStateIterator());
     verify(nmStoreSpy,times(0)).removeContainer(any(ContainerId.class));
   }
 
@@ -399,7 +482,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerRestartTimes(containerId,
         finishTimeForRetryAttempts);
     restartStateStore();
-    RecoveredContainerState rcs = stateStore.loadContainersState().get(0);
+    RecoveredContainerState rcs =
+        loadContainersState(stateStore.getContainerStateIterator()).get(0);
     List<Long> recoveredRestartTimes = rcs.getRestartTimes();
     assertEquals(1462700529039L, (long)recoveredRestartTimes.get(0));
     assertEquals(1462700529050L, (long)recoveredRestartTimes.get(1));
@@ -481,7 +565,7 @@ public class TestNMLeveldbStateStoreService {
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     RecoveredUserResources rur = userResources.get(user);
     LocalResourceTrackerState privts = rur.getPrivateTrackerState();
@@ -535,7 +619,7 @@ public class TestNMLeveldbStateStoreService {
         pubts.getInProgressResources().get(pubRsrcProto1));
     assertEquals(pubRsrcLocalPath2,
         pubts.getInProgressResources().get(pubRsrcProto2));
-    userResources = state.getUserResources();
+    userResources = loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     rur = userResources.get(user);
     privts = rur.getPrivateTrackerState();
@@ -584,7 +668,7 @@ public class TestNMLeveldbStateStoreService {
     assertTrue(pubts.getLocalizedResources().isEmpty());
     assertTrue(pubts.getInProgressResources().isEmpty());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     RecoveredUserResources rur = userResources.get(user);
     LocalResourceTrackerState privts = rur.getPrivateTrackerState();
@@ -654,7 +738,7 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(1, pubts.getInProgressResources().size());
     assertEquals(pubRsrcLocalPath2,
         pubts.getInProgressResources().get(pubRsrcProto2));
-    userResources = state.getUserResources();
+    userResources = loadUserResources(state.getIterator());
     assertEquals(1, userResources.size());
     rur = userResources.get(user);
     privts = rur.getPrivateTrackerState();
@@ -762,7 +846,7 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(pubLocalizedProto1,
         pubts.getLocalizedResources().iterator().next());
     Map<String, RecoveredUserResources> userResources =
-        state.getUserResources();
+        loadUserResources(state.getIterator());
     assertTrue(userResources.isEmpty());
   }
 
@@ -771,7 +855,9 @@ public class TestNMLeveldbStateStoreService {
     // test empty when no state
     RecoveredDeletionServiceState state =
         stateStore.loadDeletionServiceState();
-    assertTrue(state.getTasks().isEmpty());
+    List<DeletionServiceDeleteTaskProto> deleteTaskProtos =
+        loadDeletionTaskProtos(state.getIterator());
+    assertTrue(deleteTaskProtos.isEmpty());
 
     // store a deletion task and verify recovered
     DeletionServiceDeleteTaskProto proto =
@@ -788,8 +874,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeDeletionTask(proto.getId(), proto);
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertEquals(1, state.getTasks().size());
-    assertEquals(proto, state.getTasks().get(0));
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(1, deleteTaskProtos.size());
+    assertEquals(proto, deleteTaskProtos.get(0));
 
     // store another deletion task
     DeletionServiceDeleteTaskProto proto2 =
@@ -802,31 +889,36 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeDeletionTask(proto2.getId(), proto2);
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertEquals(2, state.getTasks().size());
-    assertTrue(state.getTasks().contains(proto));
-    assertTrue(state.getTasks().contains(proto2));
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(2, deleteTaskProtos.size());
+    assertTrue(deleteTaskProtos.contains(proto));
+    assertTrue(deleteTaskProtos.contains(proto2));
+
 
     // delete a task and verify gone after recovery
     stateStore.removeDeletionTask(proto2.getId());
     restartStateStore();
-    state = stateStore.loadDeletionServiceState();
-    assertEquals(1, state.getTasks().size());
-    assertEquals(proto, state.getTasks().get(0));
+    state =  stateStore.loadDeletionServiceState();
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertEquals(1, deleteTaskProtos.size());
+    assertEquals(proto, deleteTaskProtos.get(0));
 
     // delete the last task and verify none left
     stateStore.removeDeletionTask(proto.getId());
     restartStateStore();
     state = stateStore.loadDeletionServiceState();
-    assertTrue(state.getTasks().isEmpty());
-  }
+    deleteTaskProtos = loadDeletionTaskProtos(state.getIterator());
+    assertTrue(deleteTaskProtos.isEmpty());  }
 
   @Test
   public void testNMTokenStorage() throws IOException {
     // test empty when no state
     RecoveredNMTokensState state = stateStore.loadNMTokensState();
+    Map<ApplicationAttemptId, MasterKey> loadedAppKeys =
+        loadNMTokens(state.getIterator());
     assertNull(state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a master key and verify recovered
     NMTokenSecretManagerForTest secretMgr = new NMTokenSecretManagerForTest();
@@ -834,18 +926,20 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a previous key and verify recovered
     MasterKey prevKey = secretMgr.generateKey();
     stateStore.storeNMTokenPreviousMasterKey(prevKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    assertTrue(state.getApplicationMasterKeys().isEmpty());
+    assertTrue(loadedAppKeys.isEmpty());
 
     // store a few application keys and verify recovered
     ApplicationAttemptId attempt1 = ApplicationAttemptId.newInstance(
@@ -858,10 +952,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenApplicationMasterKey(attempt2, attemptKey2);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    Map<ApplicationAttemptId, MasterKey> loadedAppKeys =
-        state.getApplicationMasterKeys();
     assertEquals(2, loadedAppKeys.size());
     assertEquals(attemptKey1, loadedAppKeys.get(attempt1));
     assertEquals(attemptKey2, loadedAppKeys.get(attempt2));
@@ -880,9 +973,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeNMTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadNMTokensState();
+    loadedAppKeys = loadNMTokens(state.getIterator());
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    loadedAppKeys = state.getApplicationMasterKeys();
     assertEquals(2, loadedAppKeys.size());
     assertNull(loadedAppKeys.get(attempt1));
     assertEquals(attemptKey2, loadedAppKeys.get(attempt2));
@@ -894,9 +987,10 @@ public class TestNMLeveldbStateStoreService {
     // test empty when no state
     RecoveredContainerTokensState state =
         stateStore.loadContainerTokensState();
+    Map<ContainerId, Long> loadedActiveTokens = loadContainerTokens(state.it);
     assertNull(state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a master key and verify recovered
     ContainerTokenKeyGeneratorForTest keygen =
@@ -905,18 +999,20 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertNull(state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a previous key and verify recovered
     MasterKey prevKey = keygen.generateKey();
     stateStore.storeContainerTokenPreviousMasterKey(prevKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    assertTrue(state.getActiveTokens().isEmpty());
+    assertTrue(loadedActiveTokens.isEmpty());
 
     // store a few container tokens and verify recovered
     ContainerId cid1 = BuilderUtils.newContainerId(1, 1, 1, 1);
@@ -927,10 +1023,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerToken(cid2, expTime2);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    Map<ContainerId, Long> loadedActiveTokens =
-        state.getActiveTokens();
     assertEquals(2, loadedActiveTokens.size());
     assertEquals(expTime1, loadedActiveTokens.get(cid1));
     assertEquals(expTime2, loadedActiveTokens.get(cid2));
@@ -948,9 +1043,9 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerTokenCurrentMasterKey(currentKey);
     restartStateStore();
     state = stateStore.loadContainerTokensState();
+    loadedActiveTokens = loadContainerTokens(state.it);
     assertEquals(currentKey, state.getCurrentMasterKey());
     assertEquals(prevKey, state.getPreviousMasterKey());
-    loadedActiveTokens = state.getActiveTokens();
     assertEquals(2, loadedActiveTokens.size());
     assertNull(loadedActiveTokens.get(cid1));
     assertEquals(expTime2, loadedActiveTokens.get(cid2));
@@ -1029,8 +1124,8 @@ public class TestNMLeveldbStateStoreService {
   @Test
   public void testUnexpectedKeyDoesntThrowException() throws IOException {
     // test empty when no state
-    List<RecoveredContainerState> recoveredContainers = stateStore
-        .loadContainersState();
+    List<RecoveredContainerState> recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
@@ -1045,7 +1140,8 @@ public class TestNMLeveldbStateStoreService {
     + containerId.toString() + "/invalidKey1234").getBytes();
     stateStore.getDB().put(invalidKey, new byte[1]);
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
@@ -1162,8 +1258,8 @@ public class TestNMLeveldbStateStoreService {
   @Test
   public void testStateStoreForResourceMapping() throws IOException {
     // test empty when no state
-    List<RecoveredContainerState> recoveredContainers = stateStore
-        .loadContainersState();
+    List<RecoveredContainerState> recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertTrue(recoveredContainers.isEmpty());
 
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
@@ -1190,7 +1286,8 @@ public class TestNMLeveldbStateStoreService {
 
     // add a invalid key
     restartStateStore();
-    recoveredContainers = stateStore.loadContainersState();
+    recoveredContainers =
+        loadContainersState(stateStore.getContainerStateIterator());
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
     List<Serializable> res = rcs.getResourceMappings()
@@ -1253,7 +1350,8 @@ public class TestNMLeveldbStateStoreService {
     stateStore.storeContainerRestartTimes(containerId,
         restartTimes);
     restartStateStore();
-    RecoveredContainerState rcs = stateStore.loadContainersState().get(0);
+    RecoveredContainerState rcs =
+        loadContainersState(stateStore.getContainerStateIterator()).get(0);
     List<Long> recoveredRestartTimes = rcs.getRestartTimes();
     assertTrue(recoveredRestartTimes.isEmpty());
   }


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


[22/50] [abbrv] hadoop git commit: HADOOP-15679. ShutdownHookManager shutdown time needs to be configurable & extended. Contributed by Steve Loughran.

Posted by sh...@apache.org.
HADOOP-15679. ShutdownHookManager shutdown time needs to be configurable & extended.
Contributed by Steve Loughran.


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

Branch: refs/heads/HDFS-12943
Commit: 34577d2c21096046861d2deefdbb2638b411c687
Parents: e0f6ffd
Author: Steve Loughran <st...@apache.org>
Authored: Mon Aug 20 18:36:24 2018 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Aug 20 18:36:24 2018 -0700

----------------------------------------------------------------------
 .../fs/CommonConfigurationKeysPublic.java       |   9 +
 .../apache/hadoop/util/ShutdownHookManager.java | 169 ++++++++--
 .../src/main/resources/core-default.xml         |  16 +
 .../hadoop/util/TestShutdownHookManager.java    | 328 +++++++++++++++----
 4 files changed, 418 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index c7f32f9..b101b3b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -905,5 +905,14 @@ public class CommonConfigurationKeysPublic {
 
   public static final String HADOOP_TAGS_SYSTEM = "hadoop.tags.system";
   public static final String HADOOP_TAGS_CUSTOM = "hadoop.tags.custom";
+
+  /** Configuration option for the shutdown hook manager shutdown time:
+   *  {@value}. */
+  public static final String SERVICE_SHUTDOWN_TIMEOUT =
+      "hadoop.service.shutdown.timeout";
+
+  /** Default shutdown hook timeout: {@value} seconds. */
+  public static final long SERVICE_SHUTDOWN_TIMEOUT_DEFAULT = 30;
+
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
index 153f92b..2ca8e55 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java
@@ -17,11 +17,17 @@
  */
 package org.apache.hadoop.util;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -34,6 +40,9 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT_DEFAULT;
+
 /**
  * The <code>ShutdownHookManager</code> enables running shutdownHook
  * in a deterministic order, higher priority first.
@@ -42,53 +51,55 @@ import java.util.concurrent.atomic.AtomicBoolean;
  * This class registers a single JVM shutdownHook and run all the
  * shutdownHooks registered to it (to this class) in order based on their
  * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code core-site.xml}, with a default value of
+ * {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
  */
-public class ShutdownHookManager {
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
 
   private static final ShutdownHookManager MGR = new ShutdownHookManager();
 
   private static final Logger LOG =
       LoggerFactory.getLogger(ShutdownHookManager.class);
-  private static final long TIMEOUT_DEFAULT = 10;
-  private static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;
+
+  /** The default time unit used: seconds. */
+  public static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
 
   private static final ExecutorService EXECUTOR =
       HadoopExecutors.newSingleThreadExecutor(new ThreadFactoryBuilder()
-          .setDaemon(true).build());
+          .setDaemon(true)
+          .setNameFormat("shutdown-hook-%01d")
+          .build());
+
   static {
     try {
       Runtime.getRuntime().addShutdownHook(
         new Thread() {
           @Override
           public void run() {
-            MGR.shutdownInProgress.set(true);
-            for (HookEntry entry: MGR.getShutdownHooksInOrder()) {
-              Future<?> future = EXECUTOR.submit(entry.getHook());
-              try {
-                future.get(entry.getTimeout(), entry.getTimeUnit());
-              } catch (TimeoutException ex) {
-                future.cancel(true);
-                LOG.warn("ShutdownHook '" + entry.getHook().getClass().
-                    getSimpleName() + "' timeout, " + ex.toString(), ex);
-              } catch (Throwable ex) {
-                LOG.warn("ShutdownHook '" + entry.getHook().getClass().
-                    getSimpleName() + "' failed, " + ex.toString(), ex);
-              }
-            }
-            try {
-              EXECUTOR.shutdown();
-              if (!EXECUTOR.awaitTermination(TIMEOUT_DEFAULT,
-                  TIME_UNIT_DEFAULT)) {
-                LOG.error("ShutdownHookManger shutdown forcefully.");
-                EXECUTOR.shutdownNow();
-              }
-              LOG.debug("ShutdownHookManger complete shutdown.");
-            } catch (InterruptedException ex) {
-              LOG.error("ShutdownHookManger interrupted while waiting for " +
-                  "termination.", ex);
-              EXECUTOR.shutdownNow();
-              Thread.currentThread().interrupt();
+            if (MGR.shutdownInProgress.getAndSet(true)) {
+              LOG.info("Shutdown process invoked a second time: ignoring");
+              return;
             }
+            long started = System.currentTimeMillis();
+            int timeoutCount = executeShutdown();
+            long ended = System.currentTimeMillis();
+            LOG.debug(String.format(
+                "Completed shutdown in %.3f seconds; Timeouts: %d",
+                (ended-started)/1000.0, timeoutCount));
+            // each of the hooks have executed; now shut down the
+            // executor itself.
+            shutdownExecutor(new Configuration());
           }
         }
       );
@@ -99,18 +110,92 @@ public class ShutdownHookManager {
   }
 
   /**
+   * Execute the shutdown.
+   * This is exposed purely for testing: do not invoke it.
+   * @return the number of shutdown hooks which timed out.
+   */
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static int executeShutdown() {
+    int timeouts = 0;
+    for (HookEntry entry: MGR.getShutdownHooksInOrder()) {
+      Future<?> future = EXECUTOR.submit(entry.getHook());
+      try {
+        future.get(entry.getTimeout(), entry.getTimeUnit());
+      } catch (TimeoutException ex) {
+        timeouts++;
+        future.cancel(true);
+        LOG.warn("ShutdownHook '" + entry.getHook().getClass().
+            getSimpleName() + "' timeout, " + ex.toString(), ex);
+      } catch (Throwable ex) {
+        LOG.warn("ShutdownHook '" + entry.getHook().getClass().
+            getSimpleName() + "' failed, " + ex.toString(), ex);
+      }
+    }
+    return timeouts;
+  }
+
+  /**
+   * Shutdown the executor thread itself.
+   * @param conf the configuration containing the shutdown timeout setting.
+   */
+  private static void shutdownExecutor(final Configuration conf) {
+    try {
+      EXECUTOR.shutdown();
+      long shutdownTimeout = getShutdownTimeout(conf);
+      if (!EXECUTOR.awaitTermination(
+          shutdownTimeout,
+          TIME_UNIT_DEFAULT)) {
+        // timeout waiting for the
+        LOG.error("ShutdownHookManger shutdown forcefully after"
+            + " {} seconds.", shutdownTimeout);
+        EXECUTOR.shutdownNow();
+      }
+      LOG.debug("ShutdownHookManger completed shutdown.");
+    } catch (InterruptedException ex) {
+      // interrupted.
+      LOG.error("ShutdownHookManger interrupted while waiting for " +
+          "termination.", ex);
+      EXECUTOR.shutdownNow();
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
    * Return <code>ShutdownHookManager</code> singleton.
    *
    * @return <code>ShutdownHookManager</code> singleton.
    */
+  @InterfaceAudience.Public
   public static ShutdownHookManager get() {
     return MGR;
   }
 
   /**
+   * Get the shutdown timeout in seconds, from the supplied
+   * configuration.
+   * @param conf configuration to use.
+   * @return a timeout, always greater than or equal to {@link #TIMEOUT_MINIMUM}
+   */
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  static long getShutdownTimeout(Configuration conf) {
+    long duration = conf.getTimeDuration(
+        SERVICE_SHUTDOWN_TIMEOUT,
+        SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
+        TIME_UNIT_DEFAULT);
+    if (duration < TIMEOUT_MINIMUM) {
+      duration = TIMEOUT_MINIMUM;
+    }
+    return duration;
+  }
+
+  /**
    * Private structure to store ShutdownHook, its priority and timeout
    * settings.
    */
+  @InterfaceAudience.Private
+  @VisibleForTesting
   static class HookEntry {
     private final Runnable hook;
     private final int priority;
@@ -118,7 +203,9 @@ public class ShutdownHookManager {
     private final TimeUnit unit;
 
     HookEntry(Runnable hook, int priority) {
-      this(hook, priority, TIMEOUT_DEFAULT, TIME_UNIT_DEFAULT);
+      this(hook, priority,
+          getShutdownTimeout(new Configuration()),
+          TIME_UNIT_DEFAULT);
     }
 
     HookEntry(Runnable hook, int priority, long timeout, TimeUnit unit) {
@@ -176,10 +263,12 @@ public class ShutdownHookManager {
    *
    * @return the list of shutdownHooks in order of execution.
    */
+  @InterfaceAudience.Private
+  @VisibleForTesting
   List<HookEntry> getShutdownHooksInOrder() {
     List<HookEntry> list;
     synchronized (MGR.hooks) {
-      list = new ArrayList<HookEntry>(MGR.hooks);
+      list = new ArrayList<>(MGR.hooks);
     }
     Collections.sort(list, new Comparator<HookEntry>() {
 
@@ -200,6 +289,8 @@ public class ShutdownHookManager {
    * @param shutdownHook shutdownHook <code>Runnable</code>
    * @param priority priority of the shutdownHook.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void addShutdownHook(Runnable shutdownHook, int priority) {
     if (shutdownHook == null) {
       throw new IllegalArgumentException("shutdownHook cannot be NULL");
@@ -223,6 +314,8 @@ public class ShutdownHookManager {
    * @param timeout timeout of the shutdownHook
    * @param unit unit of the timeout <code>TimeUnit</code>
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void addShutdownHook(Runnable shutdownHook, int priority, long timeout,
       TimeUnit unit) {
     if (shutdownHook == null) {
@@ -242,6 +335,8 @@ public class ShutdownHookManager {
    * @return TRUE if the shutdownHook was registered and removed,
    * FALSE otherwise.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean removeShutdownHook(Runnable shutdownHook) {
     if (shutdownInProgress.get()) {
       throw new IllegalStateException("Shutdown in progress, cannot remove a " +
@@ -256,6 +351,8 @@ public class ShutdownHookManager {
    * @param shutdownHook shutdownHook to check if registered.
    * @return TRUE/FALSE depending if the shutdownHook is is registered.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean hasShutdownHook(Runnable shutdownHook) {
     return hooks.contains(new HookEntry(shutdownHook, 0));
   }
@@ -265,6 +362,8 @@ public class ShutdownHookManager {
    * 
    * @return TRUE if the shutdown is in progress, otherwise FALSE.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public boolean isShutdownInProgress() {
     return shutdownInProgress.get();
   }
@@ -272,7 +371,9 @@ public class ShutdownHookManager {
   /**
    * clear all registered shutdownHooks.
    */
+  @InterfaceAudience.Public
+  @InterfaceStability.Stable
   public void clearShutdownHooks() {
     hooks.clear();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 29c2bc2..7a0e1a8 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -553,6 +553,22 @@
     </description>
 </property>
 
+  <property>
+    <name>hadoop.service.shutdown.timeout</name>
+    <value>30s</value>
+    <description>
+      Timeout to wait for each shutdown operation to complete.
+      If a hook takes longer than this time to complete, it will be interrupted,
+      so the service will shutdown. This allows the service shutdown
+      to recover from a blocked operation.
+      Some shutdown hooks may need more time than this, for example when
+      a large amount of data needs to be uploaded to an object store.
+      In this situation: increase the timeout.
+
+      The minimum duration of the timeout is 1 second, "1s".
+    </description>
+</property>
+
 <property>
   <name>hadoop.rpc.protection</name>
   <value>authentication</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/34577d2c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
index d539823..03fa903 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShutdownHookManager.java
@@ -17,97 +17,285 @@
  */
 package org.apache.hadoop.util;
 
-import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.slf4j.LoggerFactory;
-import org.junit.Assert;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
 
 import static java.lang.Thread.sleep;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.SERVICE_SHUTDOWN_TIMEOUT_DEFAULT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestShutdownHookManager {
+
   static final Logger LOG =
       LoggerFactory.getLogger(TestShutdownHookManager.class.getName());
 
+  /**
+   * remove all the shutdown hooks so that they never get invoked later
+   * on in this test process.
+   */
+  @After
+  public void clearShutdownHooks() {
+    ShutdownHookManager.get().clearShutdownHooks();
+  }
+
+  /**
+   * Verify hook registration, then execute the hook callback stage
+   * of shutdown to verify invocation, execution order and timeout
+   * processing.
+   */
   @Test
   public void shutdownHookManager() {
     ShutdownHookManager mgr = ShutdownHookManager.get();
-    Assert.assertNotNull(mgr);
-    Assert.assertEquals(0, mgr.getShutdownHooksInOrder().size());
-    Runnable hook1 = new Runnable() {
-      @Override
-      public void run() {
-        LOG.info("Shutdown hook1 complete.");
-      }
-    };
-    Runnable hook2 = new Runnable() {
-      @Override
-      public void run() {
-        LOG.info("Shutdown hook2 complete.");
-      }
-    };
-
-    Runnable hook3 = new Runnable() {
-      @Override
-      public void run() {
-        try {
-          sleep(3000);
-          LOG.info("Shutdown hook3 complete.");
-        } catch (InterruptedException ex) {
-          LOG.info("Shutdown hook3 interrupted exception:",
-              ExceptionUtils.getStackTrace(ex));
-          Assert.fail("Hook 3 should not timeout.");
-        }
-      }
-    };
-
-    Runnable hook4 = new Runnable() {
-      @Override
-      public void run() {
-        try {
-          sleep(3500);
-          LOG.info("Shutdown hook4 complete.");
-          Assert.fail("Hook 4 should timeout");
-        } catch (InterruptedException ex) {
-          LOG.info("Shutdown hook4 interrupted exception:",
-              ExceptionUtils.getStackTrace(ex));
-        }
-      }
-    };
+    assertNotNull("No ShutdownHookManager", mgr);
+    assertEquals(0, mgr.getShutdownHooksInOrder().size());
+    Hook hook1 = new Hook("hook1", 0, false);
+    Hook hook2 = new Hook("hook2", 0, false);
+    Hook hook3 = new Hook("hook3", 1000, false);
+    Hook hook4 = new Hook("hook4", 25000, true);
+    Hook hook5 = new Hook("hook5",
+        (SERVICE_SHUTDOWN_TIMEOUT_DEFAULT + 1) * 1000, true);
 
     mgr.addShutdownHook(hook1, 0);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
-    Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(0).getHook());
-    mgr.removeShutdownHook(hook1);
-    Assert.assertFalse(mgr.hasShutdownHook(hook1));
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertEquals(hook1, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertTrue(mgr.removeShutdownHook(hook1));
+    assertFalse(mgr.hasShutdownHook(hook1));
+    assertFalse(mgr.removeShutdownHook(hook1));
 
     mgr.addShutdownHook(hook1, 0);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertEquals(1, mgr.getShutdownHooksInOrder().size());
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT_DEFAULT,
+        mgr.getShutdownHooksInOrder().get(0).getTimeout());
 
     mgr.addShutdownHook(hook2, 1);
-    Assert.assertTrue(mgr.hasShutdownHook(hook1));
-    Assert.assertTrue(mgr.hasShutdownHook(hook2));
-    Assert.assertEquals(2, mgr.getShutdownHooksInOrder().size());
-    Assert.assertEquals(hook2, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(hook1, mgr.getShutdownHooksInOrder().get(1).getHook());
+    assertTrue(mgr.hasShutdownHook(hook1));
+    assertTrue(mgr.hasShutdownHook(hook2));
+    assertEquals(2, mgr.getShutdownHooksInOrder().size());
+    assertEquals(hook2, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(hook1, mgr.getShutdownHooksInOrder().get(1).getHook());
 
     // Test hook finish without timeout
     mgr.addShutdownHook(hook3, 2, 4, TimeUnit.SECONDS);
-    Assert.assertTrue(mgr.hasShutdownHook(hook3));
-    Assert.assertEquals(hook3, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(4, mgr.getShutdownHooksInOrder().get(0).getTimeout());
-
-    // Test hook finish with timeout
-    mgr.addShutdownHook(hook4, 3, 2, TimeUnit.SECONDS);
-    Assert.assertTrue(mgr.hasShutdownHook(hook4));
-    Assert.assertEquals(hook4, mgr.getShutdownHooksInOrder().get(0).getHook());
-    Assert.assertEquals(2, mgr.getShutdownHooksInOrder().get(0).getTimeout());
-    LOG.info("Shutdown starts here");
+    assertTrue(mgr.hasShutdownHook(hook3));
+    assertEquals(hook3, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(4, mgr.getShutdownHooksInOrder().get(0).getTimeout());
+
+    // Test hook finish with timeout; highest priority
+    int hook4timeout = 2;
+    mgr.addShutdownHook(hook4, 3, hook4timeout, TimeUnit.SECONDS);
+    assertTrue(mgr.hasShutdownHook(hook4));
+    assertEquals(hook4, mgr.getShutdownHooksInOrder().get(0).getHook());
+    assertEquals(2, mgr.getShutdownHooksInOrder().get(0).getTimeout());
+
+    // a default timeout hook and verify it gets the default timeout
+    mgr.addShutdownHook(hook5, 5);
+    ShutdownHookManager.HookEntry hookEntry5 = mgr.getShutdownHooksInOrder()
+        .get(0);
+    assertEquals(hook5, hookEntry5.getHook());
+    assertEquals("default timeout not used",
+        ShutdownHookManager.getShutdownTimeout(new Configuration()),
+        hookEntry5.getTimeout());
+    assertEquals("hook priority", 5, hookEntry5.getPriority());
+    // remove this to avoid a longer sleep in the test run
+    assertTrue("failed to remove " + hook5,
+        mgr.removeShutdownHook(hook5));
+
+
+    // now execute the hook shutdown sequence
+    INVOCATION_COUNT.set(0);
+    LOG.info("invoking executeShutdown()");
+    int timeouts = ShutdownHookManager.executeShutdown();
+    LOG.info("Shutdown completed");
+    assertEquals("Number of timed out hooks", 1, timeouts);
+
+    List<ShutdownHookManager.HookEntry> hooks
+        = mgr.getShutdownHooksInOrder();
+
+    // analyze the hooks
+    for (ShutdownHookManager.HookEntry entry : hooks) {
+      Hook hook = (Hook) entry.getHook();
+      assertTrue("Was not invoked " + hook, hook.invoked);
+      // did any hook raise an exception?
+      hook.maybeThrowAssertion();
+    }
+
+    // check the state of some of the invoked hooks
+    // hook4 was invoked first, but it timed out.
+    assertEquals("Expected to be invoked first " + hook4,
+        1, hook4.invokedOrder);
+    assertFalse("Expected to time out " + hook4, hook4.completed);
+
+
+    // hook1 completed, but in order after the others, so its start time
+    // is the longest.
+    assertTrue("Expected to complete " + hook1, hook1.completed);
+    long invocationInterval = hook1.startTime - hook4.startTime;
+    assertTrue("invocation difference too short " + invocationInterval,
+        invocationInterval >= hook4timeout * 1000);
+    assertTrue("sleeping hook4 blocked other threads for " + invocationInterval,
+        invocationInterval < hook4.sleepTime);
+
+    // finally, clear the hooks
+    mgr.clearShutdownHooks();
+    // and verify that the hooks are empty
+    assertFalse(mgr.hasShutdownHook(hook1));
+    assertEquals("shutdown hook list is not empty",
+        0,
+        mgr.getShutdownHooksInOrder().size());
+  }
+
+  @Test
+  public void testShutdownTimeoutConfiguration() throws Throwable {
+    // set the shutdown timeout and verify it can be read back.
+    Configuration conf = new Configuration();
+    long shutdownTimeout = 5;
+    conf.setTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout, TimeUnit.SECONDS);
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout,
+        ShutdownHookManager.getShutdownTimeout(conf));
+  }
+
+  /**
+   * Verify that low timeouts simply fall back to
+   * {@link ShutdownHookManager#TIMEOUT_MINIMUM}.
+   */
+  @Test
+  public void testShutdownTimeoutBadConfiguration() throws Throwable {
+    // set the shutdown timeout and verify it can be read back.
+    Configuration conf = new Configuration();
+    long shutdownTimeout = 50;
+    conf.setTimeDuration(SERVICE_SHUTDOWN_TIMEOUT,
+        shutdownTimeout, TimeUnit.NANOSECONDS);
+    assertEquals(SERVICE_SHUTDOWN_TIMEOUT,
+        ShutdownHookManager.TIMEOUT_MINIMUM,
+        ShutdownHookManager.getShutdownTimeout(conf));
+  }
+
+  /**
+   * Verifies that a hook cannot be re-registered: an attempt to do so
+   * will simply be ignored.
+   */
+  @Test
+  public void testDuplicateRegistration() throws Throwable {
+    ShutdownHookManager mgr = ShutdownHookManager.get();
+    Hook hook = new Hook("hook1", 0, false);
+
+    // add the hook
+    mgr.addShutdownHook(hook, 2, 1, TimeUnit.SECONDS);
+
+    // add it at a higher priority. This will be ignored.
+    mgr.addShutdownHook(hook, 5);
+    List<ShutdownHookManager.HookEntry> hookList
+        = mgr.getShutdownHooksInOrder();
+    assertEquals("Hook added twice", 1, hookList.size());
+    ShutdownHookManager.HookEntry entry = hookList.get(0);
+    assertEquals("priority of hook", 2, entry.getPriority());
+    assertEquals("timeout of hook", 1, entry.getTimeout());
+
+    // remove the hook
+    assertTrue("failed to remove hook " + hook, mgr.removeShutdownHook(hook));
+    // which will fail a second time
+    assertFalse("expected hook removal to fail", mgr.removeShutdownHook(hook));
+
+    // now register it
+    mgr.addShutdownHook(hook, 5);
+    hookList = mgr.getShutdownHooksInOrder();
+    entry = hookList.get(0);
+    assertEquals("priority of hook", 5, entry.getPriority());
+    assertNotEquals("timeout of hook", 1, entry.getTimeout());
+
+  }
+
+  private static final AtomicInteger INVOCATION_COUNT = new AtomicInteger();
+
+  /**
+   * Hooks for testing; save state for ease of asserting on
+   * invocation.
+   */
+  private class Hook implements Runnable {
+
+    private final String name;
+    private final long sleepTime;
+    private final boolean expectFailure;
+    private AssertionError assertion;
+    private boolean invoked;
+    private int invokedOrder;
+    private boolean completed;
+    private boolean interrupted;
+    private long startTime;
+
+    Hook(final String name,
+        final long sleepTime,
+        final boolean expectFailure) {
+      this.name = name;
+      this.sleepTime = sleepTime;
+      this.expectFailure = expectFailure;
+    }
+
+    @Override
+    public void run() {
+      try {
+        invoked = true;
+        invokedOrder = INVOCATION_COUNT.incrementAndGet();
+        startTime = System.currentTimeMillis();
+        LOG.info("Starting shutdown of {} with sleep time of {}",
+            name, sleepTime);
+        if (sleepTime > 0) {
+          sleep(sleepTime);
+        }
+        LOG.info("Completed shutdown of {}", name);
+        completed = true;
+        if (expectFailure) {
+          assertion = new AssertionError("Expected a failure of " + name);
+        }
+      } catch (InterruptedException ex) {
+        LOG.info("Shutdown {} interrupted exception", name, ex);
+        interrupted = true;
+        if (!expectFailure) {
+          assertion = new AssertionError("Timeout of " + name, ex);
+        }
+      }
+      maybeThrowAssertion();
+    }
+
+    /**
+     * Raise any exception generated during the shutdown process.
+     * @throws AssertionError any assertion from the shutdown.
+     */
+    void maybeThrowAssertion() throws AssertionError {
+      if (assertion != null) {
+        throw assertion;
+      }
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("Hook{");
+      sb.append("name='").append(name).append('\'');
+      sb.append(", sleepTime=").append(sleepTime);
+      sb.append(", expectFailure=").append(expectFailure);
+      sb.append(", invoked=").append(invoked);
+      sb.append(", invokedOrder=").append(invokedOrder);
+      sb.append(", completed=").append(completed);
+      sb.append(", interrupted=").append(interrupted);
+      sb.append('}');
+      return sb.toString();
+    }
   }
 }


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


[24/50] [abbrv] hadoop git commit: YARN-8129. Improve error message for invalid value in fields attribute. Contributed by Abhishek Modi.

Posted by sh...@apache.org.
YARN-8129. Improve error message for invalid value in fields attribute. Contributed by Abhishek Modi.


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

Branch: refs/heads/HDFS-12943
Commit: d3fef7a5c5b83d27e87b5e49928254a7d1b935e5
Parents: 770d9d9
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Aug 21 11:58:07 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Aug 21 11:58:07 2018 +0530

----------------------------------------------------------------------
 .../timelineservice/reader/TimelineReaderWebServicesUtils.java | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fef7a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index efaecd2..63529a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -214,7 +214,11 @@ public final class TimelineReaderWebServicesUtils {
     String[] strs = str.split(delimiter);
     EnumSet<Field> fieldList = EnumSet.noneOf(Field.class);
     for (String s : strs) {
-      fieldList.add(Field.valueOf(s.trim().toUpperCase()));
+      try {
+        fieldList.add(Field.valueOf(s.trim().toUpperCase()));
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgumentException(s + " is not a valid field.");
+      }
     }
     return fieldList;
   }


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


[50/50] [abbrv] hadoop git commit: HDFS-13782. ObserverReadProxyProvider should work with IPFailoverProxyProvider. Contributed by Konstantin Shvachko.

Posted by sh...@apache.org.
HDFS-13782. ObserverReadProxyProvider should work with IPFailoverProxyProvider. Contributed by Konstantin Shvachko.

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

Branch: refs/heads/HDFS-12943
Commit: 191faeb961933c6b4fd3c3e46dd07a296607e67e
Parents: ebd3d6ab
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:32:30 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:32:30 2018 -0700

----------------------------------------------------------------------
 .../namenode/ha/ObserverReadProxyProvider.java  | 105 ++++++++++++-------
 ...ObserverReadProxyProviderWithIPFailover.java |  40 +++++++
 2 files changed, 108 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/191faeb9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java
index 754fea4..dcae2db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProvider.java
@@ -17,30 +17,30 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.ClientGSIContext;
-import org.apache.hadoop.hdfs.NameNodeProxiesClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.ipc.AlignmentContext;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * A {@link org.apache.hadoop.io.retry.FailoverProxyProvider} implementation
  * that supports reading from observer namenode(s).
@@ -55,16 +55,20 @@ import org.slf4j.LoggerFactory;
  * observer is turned off.
  */
 public class ObserverReadProxyProvider<T extends ClientProtocol>
-    extends ConfiguredFailoverProxyProvider<T> {
+    extends AbstractNNFailoverProxyProvider<T> {
   private static final Logger LOG = LoggerFactory.getLogger(
       ObserverReadProxyProvider.class);
 
   /** Client-side context for syncing with the NameNode server side */
   private AlignmentContext alignmentContext;
 
+  private AbstractNNFailoverProxyProvider<T> failoverProxy;
+  /** All NameNdoe proxies */
+  private List<NNProxyInfo<T>> nameNodeProxies =
+      new ArrayList<NNProxyInfo<T>>();
   /** Proxies for the observer namenodes */
-  private final List<AddressRpcProxyPair<T>> observerProxies =
-      new ArrayList<>();
+  private final List<NNProxyInfo<T>> observerProxies =
+      new ArrayList<NNProxyInfo<T>>();
 
   /**
    * Whether reading from observer is enabled. If this is false, all read
@@ -81,36 +85,43 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
   /** The last proxy that has been used. Only used for testing */
   private volatile ProxyInfo<T> lastProxy = null;
 
-  @SuppressWarnings("unchecked")
+  /**
+   * By default ObserverReadProxyProvider uses
+   * {@link ConfiguredFailoverProxyProvider} for failover.
+   */
   public ObserverReadProxyProvider(
       Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory)
       throws IOException {
+    this(conf, uri, xface, factory,
+        new ConfiguredFailoverProxyProvider<T>(conf, uri, xface,factory));
+  }
+
+  public ObserverReadProxyProvider(
+      Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory,
+      AbstractNNFailoverProxyProvider<T> failoverProxy)
+      throws IOException {
     super(conf, uri, xface, factory);
-    alignmentContext = new ClientGSIContext();
+    this.failoverProxy = failoverProxy;
+    this.alignmentContext = new ClientGSIContext();
     ((ClientHAProxyFactory<T>) factory).setAlignmentContext(alignmentContext);
 
+    // Get all NameNode proxies
+    nameNodeProxies = getProxyAddresses(uri,
+        HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
     // Find out all the observer proxies
-    for (AddressRpcProxyPair<T> ap : this.proxies) {
-      ap.namenode = (T) NameNodeProxiesClient.createProxyWithAlignmentContext(
-          ap.address, conf, ugi, false, getFallbackToSimpleAuth(),
-          alignmentContext);
-      if (isObserverState(ap)) {
-        observerProxies.add(ap);
+    for (NNProxyInfo<T> pi : nameNodeProxies) {
+      createProxyIfNeeded(pi);
+      if (isObserverState(pi)) {
+        observerProxies.add(pi);
       }
     }
 
+    // TODO: No observers is not an error
+    // Just direct all reads go to the active NameNode
     if (observerProxies.isEmpty()) {
       throw new RuntimeException("Couldn't find any namenode proxy in " +
           "OBSERVER state");
     }
-
-    // Randomize the list to prevent all clients pointing to the same one
-    boolean randomized = conf.getBoolean(
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER,
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
-    if (randomized) {
-      Collections.shuffle(observerProxies);
-    }
   }
 
   public synchronized AlignmentContext getAlignmentContext() {
@@ -121,17 +132,13 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
   @Override
   public synchronized ProxyInfo<T> getProxy() {
     // We just create a wrapped proxy containing all the proxies
-    List<ProxyInfo<T>> observerProxies = new ArrayList<>();
     StringBuilder combinedInfo = new StringBuilder("[");
 
     for (int i = 0; i < this.observerProxies.size(); i++) {
       if (i > 0) {
         combinedInfo.append(",");
       }
-      AddressRpcProxyPair<T> p = this.observerProxies.get(i);
-      ProxyInfo<T> pInfo = getProxy(p);
-      observerProxies.add(pInfo);
-      combinedInfo.append(pInfo.proxyInfo);
+      combinedInfo.append(observerProxies.get(i).proxyInfo);
     }
 
     combinedInfo.append(']');
@@ -142,6 +149,11 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
     return new ProxyInfo<>(wrappedProxy, combinedInfo.toString());
   }
 
+  @Override
+  public void performFailover(T currentProxy) {
+    failoverProxy.performFailover(currentProxy);
+  }
+
   /**
    * Check if a method is read-only.
    *
@@ -170,14 +182,14 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
     return lastProxy;
   }
 
-  boolean isObserverState(AddressRpcProxyPair<T> ap) {
+  boolean isObserverState(NNProxyInfo<T> pi) {
     // TODO: should introduce new ClientProtocol method to verify the
     // underlying service state, which does not require superuser access
     // The is a workaround
     IOException ioe = null;
     try {
       // Verify write access first
-      ap.namenode.reportBadBlocks(new LocatedBlock[0]);
+      pi.proxy.reportBadBlocks(new LocatedBlock[0]);
       return false; // Only active NameNode allows write
     } catch (RemoteException re) {
       IOException sbe = re.unwrapRemoteException(StandbyException.class);
@@ -188,14 +200,14 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
       ioe = e;
     }
     if (ioe != null) {
-      LOG.error("Failed to connect to {}", ap.address, ioe);
+      LOG.error("Failed to connect to {}", pi.getAddress(), ioe);
       return false;
     }
     // Verify read access
     // For now we assume only Observer nodes allow reads
     // Stale reads on StandbyNode should be turned off
     try {
-      ap.namenode.checkAccess("/", FsAction.READ);
+      pi.proxy.checkAccess("/", FsAction.READ);
       return true;
     } catch (RemoteException re) {
       IOException sbe = re.unwrapRemoteException(StandbyException.class);
@@ -206,19 +218,19 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
       ioe = e;
     }
     if (ioe != null) {
-      LOG.error("Failed to connect to {}", ap.address, ioe);
+      LOG.error("Failed to connect to {}", pi.getAddress(), ioe);
     }
     return false;
   }
 
 
   class ObserverReadInvocationHandler implements InvocationHandler {
-    final List<ProxyInfo<T>> observerProxies;
+    final List<NNProxyInfo<T>> observerProxies;
     final ProxyInfo<T> activeProxy;
 
-    ObserverReadInvocationHandler(List<ProxyInfo<T>> observerProxies) {
+    ObserverReadInvocationHandler(List<NNProxyInfo<T>> observerProxies) {
       this.observerProxies = observerProxies;
-      this.activeProxy = ObserverReadProxyProvider.super.getProxy();
+      this.activeProxy = failoverProxy.getProxy();
     }
 
     /**
@@ -238,7 +250,7 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
       if (observerReadEnabled && isRead(method)) {
         // Loop through all the proxies, starting from the current index.
         for (int i = 0; i < observerProxies.size(); i++) {
-          ProxyInfo<T> current = observerProxies.get(currentIndex.get());
+          NNProxyInfo<T> current = observerProxies.get(currentIndex.get());
           try {
             retVal = method.invoke(current.proxy, args);
             lastProxy = current;
@@ -269,4 +281,23 @@ public class ObserverReadProxyProvider<T extends ClientProtocol>
       return retVal;
     }
   }
+
+  @Override
+  public synchronized void close() throws IOException {
+    failoverProxy.close();
+    for (ProxyInfo<T> pi : nameNodeProxies) {
+      if (pi.proxy != null) {
+        if (pi.proxy instanceof Closeable) {
+          ((Closeable)pi.proxy).close();
+        } else {
+          RPC.stopProxy(pi.proxy);
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean useLogicalURI() {
+    return failoverProxy.useLogicalURI();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/191faeb9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProviderWithIPFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProviderWithIPFailover.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProviderWithIPFailover.java
new file mode 100644
index 0000000..1dbd02c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ObserverReadProxyProviderWithIPFailover.java
@@ -0,0 +1,40 @@
+/**
+ * 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.hdfs.server.namenode.ha;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+
+/**
+ * ObserverReadProxyProvider with IPFailoverProxyProvider
+ * as the failover method.
+ */
+public class
+ObserverReadProxyProviderWithIPFailover<T extends ClientProtocol>
+extends ObserverReadProxyProvider<T> {
+
+  public ObserverReadProxyProviderWithIPFailover(
+      Configuration conf, URI uri, Class<T> xface,
+      HAProxyFactory<T> factory) throws IOException {
+    super(conf, uri, xface, factory,
+        new IPFailoverProxyProvider<T>(conf, uri, xface,factory));
+  }
+}
\ No newline at end of file


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


[48/50] [abbrv] hadoop git commit: Merge commit '96c4575d7373079becfa3e3db29ba98e6fb86388' into HDFS-12943

Posted by sh...@apache.org.
Merge commit '96c4575d7373079becfa3e3db29ba98e6fb86388' into HDFS-12943

# Conflicts:
#	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java


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

Branch: refs/heads/HDFS-12943
Commit: 8aeb2d8942116ec924f319569949516055f3f5c2
Parents: da18227 96c4575
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:28:33 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:28:33 2018 -0700

----------------------------------------------------------------------
 dev-support/byteman/README.md                   |  31 +
 dev-support/byteman/hadooprpc.btm               |  44 ++
 .../fs/CommonConfigurationKeysPublic.java       |   9 +
 .../apache/hadoop/util/ShutdownHookManager.java | 169 ++++-
 .../src/main/resources/core-default.xml         |  16 +
 .../site/markdown/InterfaceClassification.md    |   7 +-
 .../hadoop/util/TestShutdownHookManager.java    | 328 ++++++--
 .../src/main/compose/ozone/docker-config        |   2 +
 .../container/common/helpers/ContainerInfo.java |   1 +
 .../org/apache/hadoop/utils/db/DBStore.java     |  93 +++
 .../org/apache/hadoop/utils/db/RDBStore.java    | 252 +++++++
 .../hadoop/utils/db/RDBStoreIterator.java       |  88 +++
 .../org/apache/hadoop/utils/db/RDBTable.java    | 173 +++++
 .../java/org/apache/hadoop/utils/db/Table.java  | 150 ++++
 .../apache/hadoop/utils/db/TableIterator.java   |  50 ++
 .../apache/hadoop/utils/db/package-info.java    |  22 +
 .../apache/hadoop/utils/db/TestRDBStore.java    | 246 ++++++
 .../hadoop/utils/db/TestRDBTableStore.java      | 189 +++++
 .../apache/hadoop/utils/db/package-info.java    |  22 +
 .../apache/hadoop/hdds/scm/HddsServerUtil.java  |   6 +-
 .../common/helpers/ContainerReport.java         |  14 -
 .../common/helpers/KeyValueContainerReport.java | 117 +++
 .../container/common/impl/ContainerData.java    |  53 --
 .../common/impl/ContainerDataYaml.java          |  94 ++-
 .../container/common/impl/ContainerSet.java     |  50 +-
 .../RandomContainerDeletionChoosingPolicy.java  |   3 +-
 ...NOrderedContainerDeletionChoosingPolicy.java |  24 +-
 .../container/common/interfaces/Container.java  |  39 +-
 .../ContainerDeletionChoosingPolicy.java        |  13 +
 .../common/interfaces/ContainerPacker.java      |  58 ++
 .../report/CommandStatusReportPublisher.java    |   2 +-
 .../common/report/ContainerReportPublisher.java |   2 +-
 .../common/report/NodeReportPublisher.java      |   2 +-
 .../statemachine/DatanodeStateMachine.java      |   3 +-
 .../statemachine/EndpointStateMachine.java      |   9 +-
 .../container/keyvalue/KeyValueContainer.java   | 174 ++++-
 .../keyvalue/KeyValueContainerData.java         |  54 ++
 .../container/keyvalue/KeyValueHandler.java     |  19 +-
 .../container/keyvalue/TarContainerPacker.java  | 249 +++++++
 .../keyvalue/helpers/KeyValueContainerUtil.java |  22 +-
 .../common/TestKeyValueContainerData.java       |   4 +-
 .../keyvalue/TestKeyValueContainer.java         |  95 ++-
 .../keyvalue/TestTarContainerPacker.java        | 231 ++++++
 .../hdds/scm/container/ContainerMapping.java    |  23 +-
 .../scm/container/closer/ContainerCloser.java   |   9 +-
 .../scm/container/TestContainerMapping.java     |   8 +
 .../federation/resolver/MountTableResolver.java |  37 +-
 .../server/federation/router/RBFConfigKeys.java |   4 +
 .../federation/router/RouterRpcClient.java      |  18 +-
 .../federation/router/RouterRpcServer.java      |   3 +-
 .../src/main/resources/hdfs-rbf-default.xml     |   9 +
 .../hdfs/server/federation/MockResolver.java    |   3 +
 .../resolver/TestMountTableResolver.java        |  31 +
 .../server/federation/router/TestRouter.java    |  18 +
 .../hdfs/qjournal/client/AsyncLogger.java       |   3 +-
 .../hdfs/qjournal/client/AsyncLoggerSet.java    |   4 +-
 .../hdfs/qjournal/client/IPCLoggerChannel.java  |   5 +-
 .../qjournal/client/QuorumJournalManager.java   |   4 +-
 .../qjournal/protocol/QJournalProtocol.java     |   2 +-
 .../QJournalProtocolServerSideTranslatorPB.java |   2 +-
 .../QJournalProtocolTranslatorPB.java           |   6 +-
 .../hadoop/hdfs/qjournal/server/JNStorage.java  |   4 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |   6 +-
 .../qjournal/server/JournalNodeRpcServer.java   |   5 +-
 .../server/namenode/BackupJournalManager.java   |   2 +-
 .../namenode/ErasureCodingPolicyManager.java    |  15 +-
 .../server/namenode/FSDirErasureCodingOp.java   |  22 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   5 +-
 .../hadoop/hdfs/server/namenode/FSImage.java    |   5 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  30 +-
 .../server/namenode/FileJournalManager.java     |   2 +-
 .../hdfs/server/namenode/JournalManager.java    |   2 +-
 .../hadoop/hdfs/server/namenode/JournalSet.java |   2 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   4 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   8 +-
 .../src/main/proto/QJournalProtocol.proto       |   1 +
 .../qjournal/client/TestEpochsAreUnique.java    |   2 +-
 .../hdfs/qjournal/client/TestQJMWithFaults.java |   6 +-
 .../client/TestQuorumJournalManager.java        |   4 +-
 .../client/TestQuorumJournalManagerUnit.java    |   4 +-
 .../hdfs/qjournal/server/TestJournal.java       |  17 +-
 .../hdfs/qjournal/server/TestJournalNode.java   |   4 +-
 .../qjournal/server/TestJournalNodeMXBean.java  |   2 +-
 .../qjournal/server/TestJournalNodeSync.java    |   2 +-
 .../server/namenode/TestGenericJournalConf.java |   2 +-
 .../server/namenode/TestNamenodeRetryCache.java |   2 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |   2 +-
 .../ozone/TestStorageContainerManager.java      |  19 +-
 .../common/TestBlockDeletingService.java        |   8 +-
 .../commandhandler/TestBlockDeletion.java       |  19 +-
 .../hadoop/ozone/web/client/TestKeys.java       |  10 +-
 .../contract/AbstractContractDistCpTest.java    |   2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |   2 +
 .../yarn/service/client/ApiServiceClient.java   |  20 +
 .../hadoop/yarn/service/webapp/ApiServer.java   |  12 +-
 .../hadoop/yarn/service/ClientAMService.java    |   2 +-
 .../hadoop/yarn/service/ServiceEvent.java       |  25 +
 .../hadoop/yarn/service/ServiceManager.java     | 127 +++-
 .../hadoop/yarn/service/ServiceScheduler.java   |  15 +-
 .../yarn/service/api/records/ServiceState.java  |   2 +-
 .../yarn/service/client/ServiceClient.java      | 100 ++-
 .../yarn/service/component/Component.java       |  16 +-
 .../yarn/service/component/ComponentEvent.java  |  10 +
 .../component/instance/ComponentInstance.java   |   5 +
 .../yarn/service/utils/ServiceApiUtil.java      |  44 ++
 .../src/main/proto/ClientAMProtocol.proto       |   1 +
 .../hadoop/yarn/service/TestServiceApiUtil.java | 653 ----------------
 .../hadoop/yarn/service/TestServiceManager.java | 299 +++++---
 .../yarn/service/TestYarnNativeServices.java    |  35 +
 .../yarn/service/utils/TestServiceApiUtil.java  | 743 +++++++++++++++++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  20 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   4 +
 .../hadoop/yarn/client/AMRMClientUtils.java     |  47 ++
 .../hadoop/yarn/client/api/AppAdminClient.java  |  12 +
 .../hadoop/yarn/server/AMRMClientRelayer.java   | 130 ++--
 .../LocalityMulticastAMRMProxyPolicy.java       |  64 +-
 .../utils/FederationStateStoreFacade.java       |   9 +
 .../server/uam/UnmanagedApplicationManager.java |   2 +-
 .../yarn/server/TestAMRMClientRelayer.java      |  53 +-
 .../TestLocalityMulticastAMRMProxyPolicy.java   |  91 ++-
 .../utils/FederationPoliciesTestUtil.java       |   7 +-
 .../server/nodemanager/DeletionService.java     |  25 +-
 .../amrmproxy/FederationInterceptor.java        |   6 +-
 .../containermanager/ContainerManagerImpl.java  |  26 +-
 .../localizer/LocalResourcesTrackerImpl.java    |   5 +
 .../localizer/ResourceLocalizationService.java  |  68 +-
 .../recovery/NMLeveldbStateStoreService.java    | 412 ++++++----
 .../recovery/NMNullStateStoreService.java       |   2 +-
 .../recovery/NMStateStoreService.java           |  55 +-
 .../nodemanager/recovery/RecoveryIterator.java  |  41 +
 .../security/NMContainerTokenSecretManager.java |  27 +-
 .../security/NMTokenSecretManagerInNM.java      |  15 +-
 .../TestResourceLocalizationService.java        |  12 +-
 .../recovery/NMMemoryStateStoreService.java     |  82 +-
 .../TestNMLeveldbStateStoreService.java         | 216 ++++--
 .../ApplicationMasterService.java               |   9 +-
 .../resourcemanager/RMActiveServiceContext.java |  16 +
 .../yarn/server/resourcemanager/RMContext.java  |   8 +-
 .../server/resourcemanager/RMContextImpl.java   |  14 +-
 .../server/resourcemanager/ResourceManager.java |  12 +
 .../scheduler/AbstractYarnScheduler.java        |  10 +
 .../scheduler/AppSchedulingInfo.java            |  28 +-
 .../scheduler/ClusterNodeTracker.java           |  61 ++
 .../scheduler/YarnScheduler.java                |  10 +
 .../scheduler/activities/ActivitiesLogger.java  |  32 +-
 .../scheduler/activities/ActivitiesManager.java |   8 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  16 +-
 .../scheduler/capacity/CSQueue.java             |   6 +
 .../scheduler/capacity/CapacityScheduler.java   |  77 +-
 .../CapacitySchedulerConfiguration.java         | 116 +++
 .../scheduler/capacity/LeafQueue.java           |  49 +-
 .../scheduler/capacity/ParentQueue.java         |   4 +-
 .../allocator/RegularContainerAllocator.java    |  35 +-
 .../common/ApplicationSchedulingConfig.java     |   4 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |  23 +
 .../placement/AppPlacementAllocator.java        |   6 +
 .../LocalityAppPlacementAllocator.java          |  39 +-
 .../placement/MultiNodeLookupPolicy.java        |  67 ++
 .../placement/MultiNodePolicySpec.java          |  56 ++
 .../scheduler/placement/MultiNodeSorter.java    | 167 +++++
 .../placement/MultiNodeSortingManager.java      | 139 ++++
 .../ResourceUsageMultiNodeLookupPolicy.java     |  79 ++
 .../SingleConstraintAppPlacementAllocator.java  | 179 ++---
 .../webapp/RMAppAttemptBlock.java               |  24 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |  10 +
 .../webapp/dao/ResourceRequestInfo.java         |  52 +-
 .../yarn/server/resourcemanager/MockRM.java     |  35 +-
 .../reservation/ReservationSystemTestUtil.java  |   3 +
 .../scheduler/TestAppSchedulingInfo.java        |   3 +-
 .../capacity/CapacitySchedulerTestBase.java     |  13 +
 .../capacity/TestCapacityScheduler.java         |  15 -
 .../TestCapacitySchedulerMultiNodes.java        | 166 +++++
 .../TestCapacitySchedulerNodeLabelUpdate.java   |  70 ++
 ...estSchedulingRequestContainerAllocation.java | 438 ++++++++++-
 ...stSingleConstraintAppPlacementAllocator.java |  78 --
 .../reader/TimelineReaderWebServicesUtils.java  |   6 +-
 177 files changed, 7363 insertions(+), 2006 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
index b38a527,9e1e3bb..dd55af2
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
@@@ -17,13 -17,11 +17,14 @@@
   */
  package org.apache.hadoop.hdfs.qjournal.client;
  
 +import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.fail;
  import static org.mockito.Matchers.anyLong;
+ import static org.mockito.Matchers.anyBoolean;
  import static org.mockito.Matchers.eq;
  
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataOutputStream;
  import java.io.IOException;
  import java.net.URI;
  import java.util.List;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aeb2d89/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
index d419f76,b8d2652..2f51275
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
@@@ -78,11 -71,9 +78,11 @@@ public class TestJournal 
    public void setup() throws Exception {
      FileUtil.fullyDelete(TEST_LOG_DIR);
      conf = new Configuration();
 +    // Enable fetching edits via RPC
 +    conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
      journal = new Journal(conf, TEST_LOG_DIR, JID, StartupOption.REGULAR,
        mockErrorReporter);
-     journal.format(FAKE_NSINFO);
+     journal.format(FAKE_NSINFO, false);
    }
    
    @After
@@@ -444,41 -435,14 +444,52 @@@
    }
  
    @Test
 +  public void testReadFromCache() throws Exception {
 +    journal.newEpoch(FAKE_NSINFO, 1);
 +    journal.startLogSegment(makeRI(1), 1,
 +        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 +    journal.journal(makeRI(2), 1, 1, 5, QJMTestUtil.createTxnData(1, 5));
 +    journal.journal(makeRI(3), 1, 6, 5, QJMTestUtil.createTxnData(6, 5));
 +    journal.journal(makeRI(4), 1, 11, 5, QJMTestUtil.createTxnData(11, 5));
 +    assertJournaledEditsTxnCountAndContents(1, 7, 7,
 +        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 +    assertJournaledEditsTxnCountAndContents(1, 30, 15,
 +        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 +
 +    journal.finalizeLogSegment(makeRI(5), 1, 15);
 +    int newLayoutVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
 +    journal.startLogSegment(makeRI(6), 16, newLayoutVersion);
 +    journal.journal(makeRI(7), 16, 16, 5, QJMTestUtil.createTxnData(16, 5));
 +
 +    assertJournaledEditsTxnCountAndContents(16, 10, 20, newLayoutVersion);
 +  }
 +
 +  private void assertJournaledEditsTxnCountAndContents(int startTxn,
 +      int requestedMaxTxns, int expectedEndTxn, int layoutVersion)
 +      throws Exception {
 +    GetJournaledEditsResponseProto result =
 +        journal.getJournaledEdits(startTxn, requestedMaxTxns);
 +    int expectedTxnCount = expectedEndTxn - startTxn + 1;
 +    ByteArrayOutputStream headerBytes = new ByteArrayOutputStream();
 +    EditLogFileOutputStream.writeHeader(layoutVersion,
 +        new DataOutputStream(headerBytes));
 +    assertEquals(expectedTxnCount, result.getTxnCount());
 +    assertArrayEquals(
 +        Bytes.concat(
 +            headerBytes.toByteArray(),
 +            QJMTestUtil.createTxnData(startTxn, expectedTxnCount)),
 +        result.getEditLog().toByteArray());
 +  }
 +
++  @Test
+   public void testFormatNonEmptyStorageDirectoriesWhenforceOptionIsTrue()
+       throws Exception {
+     try {
+       // Format again here and to format the non-empty directories in
+       // journal node.
+       journal.format(FAKE_NSINFO, true);
+     } catch (IOException ioe) {
+       fail("Format should be success with force option.");
+     }
+   }
  }


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


[29/50] [abbrv] hadoop git commit: YARN-8298. Added express upgrade for YARN service. Contributed by Chandni Singh

Posted by sh...@apache.org.
YARN-8298.  Added express upgrade for YARN service.
            Contributed by Chandni Singh


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

Branch: refs/heads/HDFS-12943
Commit: e557c6bd8de2811a561210f672f47b4d07a9d5c6
Parents: 9c3fc3e
Author: Eric Yang <ey...@apache.org>
Authored: Tue Aug 21 19:49:26 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Aug 21 19:49:26 2018 -0400

----------------------------------------------------------------------
 .../yarn/service/client/ApiServiceClient.java   |  20 +
 .../hadoop/yarn/service/webapp/ApiServer.java   |  12 +-
 .../hadoop/yarn/service/ClientAMService.java    |   2 +-
 .../hadoop/yarn/service/ServiceEvent.java       |  25 +
 .../hadoop/yarn/service/ServiceManager.java     | 127 +++-
 .../hadoop/yarn/service/ServiceScheduler.java   |  15 +-
 .../yarn/service/api/records/ServiceState.java  |   2 +-
 .../yarn/service/client/ServiceClient.java      | 100 ++-
 .../yarn/service/component/Component.java       |  16 +-
 .../yarn/service/component/ComponentEvent.java  |  10 +
 .../component/instance/ComponentInstance.java   |   5 +
 .../yarn/service/utils/ServiceApiUtil.java      |  44 ++
 .../src/main/proto/ClientAMProtocol.proto       |   1 +
 .../hadoop/yarn/service/TestServiceApiUtil.java | 653 ----------------
 .../hadoop/yarn/service/TestServiceManager.java | 299 +++++---
 .../yarn/service/TestYarnNativeServices.java    |  35 +
 .../yarn/service/utils/TestServiceApiUtil.java  | 743 +++++++++++++++++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |  20 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   4 +
 .../hadoop/yarn/client/api/AppAdminClient.java  |  12 +
 20 files changed, 1308 insertions(+), 837 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
index 9229446..ca6cc50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
@@ -601,6 +601,26 @@ public class ApiServiceClient extends AppAdminClient {
   }
 
   @Override
+  public int actionUpgradeExpress(String appName, File path)
+      throws IOException, YarnException {
+    int result;
+    try {
+      Service service =
+          loadAppJsonFromLocalFS(path.getAbsolutePath(), appName, null, null);
+      service.setState(ServiceState.EXPRESS_UPGRADING);
+      String buffer = jsonSerDeser.toJson(service);
+      LOG.info("Upgrade in progress. Please wait..");
+      ClientResponse response = getApiClient(getServicePath(appName))
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Failed to upgrade application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  @Override
   public int initiateUpgrade(String appName,
       String fileName, boolean autoFinalize) throws IOException, YarnException {
     int result;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 4db0ac8..cd6f0d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -440,7 +440,8 @@ public class ApiServer {
       if (updateServiceData.getState() != null && (
           updateServiceData.getState() == ServiceState.UPGRADING ||
               updateServiceData.getState() ==
-                  ServiceState.UPGRADING_AUTO_FINALIZE)) {
+                  ServiceState.UPGRADING_AUTO_FINALIZE) ||
+          updateServiceData.getState() == ServiceState.EXPRESS_UPGRADING) {
         return upgradeService(updateServiceData, ugi);
       }
 
@@ -690,7 +691,11 @@ public class ApiServer {
       ServiceClient sc = getServiceClient();
       sc.init(YARN_CONFIG);
       sc.start();
-      sc.initiateUpgrade(service);
+      if (service.getState().equals(ServiceState.EXPRESS_UPGRADING)) {
+        sc.actionUpgradeExpress(service);
+      } else {
+        sc.initiateUpgrade(service);
+      }
       sc.close();
       return null;
     });
@@ -706,7 +711,8 @@ public class ApiServer {
       String serviceName, Set<String> compNames) throws YarnException,
       IOException, InterruptedException {
     Service service = getServiceFromClient(ugi, serviceName);
-    if (service.getState() != ServiceState.UPGRADING) {
+    if (!service.getState().equals(ServiceState.UPGRADING) &&
+        !service.getState().equals(ServiceState.UPGRADING_AUTO_FINALIZE)) {
       throw new YarnException(
           String.format("The upgrade of service %s has not been initiated.",
               service.getName()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
index 5bf1833..2ef8f7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ClientAMService.java
@@ -166,7 +166,7 @@ public class ClientAMService extends AbstractService
       LOG.info("Upgrading service to version {} by {}", request.getVersion(),
           UserGroupInformation.getCurrentUser());
       context.getServiceManager().processUpgradeRequest(request.getVersion(),
-          request.getAutoFinalize());
+          request.getAutoFinalize(), request.getExpressUpgrade());
       return UpgradeServiceResponseProto.newBuilder().build();
     } catch (Exception ex) {
       return UpgradeServiceResponseProto.newBuilder().setError(ex.getMessage())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.java
index 0196be2..3a55472 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceEvent.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.service;
 
 import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.hadoop.yarn.service.api.records.Component;
+
+import java.util.Queue;
 
 /**
  * Events are handled by {@link ServiceManager} to manage the service
@@ -29,6 +32,8 @@ public class ServiceEvent extends AbstractEvent<ServiceEventType> {
   private final ServiceEventType type;
   private String version;
   private boolean autoFinalize;
+  private boolean expressUpgrade;
+  private Queue<Component> compsToUpgradeInOrder;
 
   public ServiceEvent(ServiceEventType serviceEventType) {
     super(serviceEventType);
@@ -56,4 +61,24 @@ public class ServiceEvent extends AbstractEvent<ServiceEventType> {
     this.autoFinalize = autoFinalize;
     return this;
   }
+
+  public boolean isExpressUpgrade() {
+    return expressUpgrade;
+  }
+
+  public ServiceEvent setExpressUpgrade(boolean expressUpgrade) {
+    this.expressUpgrade = expressUpgrade;
+    return this;
+  }
+
+  public Queue<Component> getCompsToUpgradeInOrder() {
+    return compsToUpgradeInOrder;
+  }
+
+  public ServiceEvent setCompsToUpgradeInOrder(
+      Queue<Component> compsToUpgradeInOrder) {
+    this.compsToUpgradeInOrder = compsToUpgradeInOrder;
+    return this;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.java
index 05ecb3f..04454b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceManager.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.Component;
@@ -40,8 +41,11 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.text.MessageFormat;
 import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.jsonSerDeser;
@@ -67,6 +71,8 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
   private final SliderFileSystem fs;
 
   private String upgradeVersion;
+  private Queue<org.apache.hadoop.yarn.service.api.records
+        .Component> compsToUpgradeInOrder;
 
   private static final StateMachineFactory<ServiceManager, State,
       ServiceEventType, ServiceEvent> STATE_MACHINE_FACTORY =
@@ -141,14 +147,20 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
     @Override
     public State transition(ServiceManager serviceManager,
         ServiceEvent event) {
+      serviceManager.upgradeVersion = event.getVersion();
       try {
-        if (!event.isAutoFinalize()) {
-          serviceManager.serviceSpec.setState(ServiceState.UPGRADING);
+        if (event.isExpressUpgrade()) {
+          serviceManager.serviceSpec.setState(ServiceState.EXPRESS_UPGRADING);
+          serviceManager.compsToUpgradeInOrder = event
+              .getCompsToUpgradeInOrder();
+          serviceManager.upgradeNextCompIfAny();
+        } else if (event.isAutoFinalize()) {
+          serviceManager.serviceSpec.setState(ServiceState
+              .UPGRADING_AUTO_FINALIZE);
         } else {
           serviceManager.serviceSpec.setState(
-              ServiceState.UPGRADING_AUTO_FINALIZE);
+              ServiceState.UPGRADING);
         }
-        serviceManager.upgradeVersion = event.getVersion();
         return State.UPGRADING;
       } catch (Throwable e) {
         LOG.error("[SERVICE]: Upgrade to version {} failed", event.getVersion(),
@@ -169,8 +181,19 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
       if (currState.equals(ServiceState.STABLE)) {
         return State.STABLE;
       }
+      if (currState.equals(ServiceState.EXPRESS_UPGRADING)) {
+        org.apache.hadoop.yarn.service.api.records.Component component =
+            serviceManager.compsToUpgradeInOrder.peek();
+        if (!component.getState().equals(ComponentState.NEEDS_UPGRADE) &&
+            !component.getState().equals(ComponentState.UPGRADING)) {
+          serviceManager.compsToUpgradeInOrder.remove();
+        }
+        serviceManager.upgradeNextCompIfAny();
+      }
       if (currState.equals(ServiceState.UPGRADING_AUTO_FINALIZE) ||
-          event.getType().equals(ServiceEventType.START)) {
+          event.getType().equals(ServiceEventType.START) ||
+          (currState.equals(ServiceState.EXPRESS_UPGRADING) &&
+              serviceManager.compsToUpgradeInOrder.isEmpty())) {
         ServiceState targetState = checkIfStable(serviceManager.serviceSpec);
         if (targetState.equals(ServiceState.STABLE)) {
           if (serviceManager.finalizeUpgrade()) {
@@ -184,6 +207,19 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
     }
   }
 
+  private void upgradeNextCompIfAny() {
+    if (!compsToUpgradeInOrder.isEmpty()) {
+      org.apache.hadoop.yarn.service.api.records.Component component =
+          compsToUpgradeInOrder.peek();
+
+      ComponentEvent needUpgradeEvent = new ComponentEvent(
+          component.getName(), ComponentEventType.UPGRADE).setTargetSpec(
+          component).setUpgradeVersion(upgradeVersion).setExpressUpgrade(true);
+      context.scheduler.getDispatcher().getEventHandler().handle(
+          needUpgradeEvent);
+    }
+  }
+
   /**
    * @return whether finalization of upgrade was successful.
    */
@@ -250,23 +286,18 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
   }
 
   void processUpgradeRequest(String upgradeVersion,
-      boolean autoFinalize) throws IOException {
+      boolean autoFinalize, boolean expressUpgrade) throws IOException {
     Service targetSpec = ServiceApiUtil.loadServiceUpgrade(
         context.fs, context.service.getName(), upgradeVersion);
 
     List<org.apache.hadoop.yarn.service.api.records.Component>
-        compsThatNeedUpgrade = componentsFinder.
+        compsNeedUpgradeList = componentsFinder.
         findTargetComponentSpecs(context.service, targetSpec);
-    ServiceEvent event = new ServiceEvent(ServiceEventType.UPGRADE)
-        .setVersion(upgradeVersion)
-        .setAutoFinalize(autoFinalize);
-    context.scheduler.getDispatcher().getEventHandler().handle(event);
 
-    if (compsThatNeedUpgrade != null && !compsThatNeedUpgrade.isEmpty()) {
-      if (autoFinalize) {
-        event.setAutoFinalize(true);
-      }
-      compsThatNeedUpgrade.forEach(component -> {
+    // remove all components from need upgrade list if there restart policy
+    // doesn't all upgrade.
+    if (compsNeedUpgradeList != null) {
+      compsNeedUpgradeList.removeIf(component -> {
         org.apache.hadoop.yarn.service.api.records.Component.RestartPolicyEnum
             restartPolicy = component.getRestartPolicy();
 
@@ -274,25 +305,65 @@ public class ServiceManager implements EventHandler<ServiceEvent> {
             Component.getRestartPolicyHandler(restartPolicy);
         // Do not allow upgrades for components which have NEVER/ON_FAILURE
         // restart policy
-        if (restartPolicyHandler.allowUpgrades()) {
+        if (!restartPolicyHandler.allowUpgrades()) {
+          LOG.info("The component {} has a restart policy that doesnt " +
+                  "allow upgrades {} ", component.getName(),
+              component.getRestartPolicy().toString());
+          return true;
+        }
+
+        return false;
+      });
+    }
+
+    ServiceEvent event = new ServiceEvent(ServiceEventType.UPGRADE)
+        .setVersion(upgradeVersion)
+        .setAutoFinalize(autoFinalize)
+        .setExpressUpgrade(expressUpgrade);
+
+    if (expressUpgrade) {
+      // In case of express upgrade  components need to be upgraded in order.
+      // Once the service manager gets notified that a component finished
+      // upgrading, it then issues event to upgrade the next component.
+      Map<String, org.apache.hadoop.yarn.service.api.records.Component>
+          compsNeedUpgradeByName = new HashMap<>();
+      if (compsNeedUpgradeList != null) {
+        compsNeedUpgradeList.forEach(component ->
+            compsNeedUpgradeByName.put(component.getName(), component));
+      }
+      List<String> resolvedComps = ServiceApiUtil
+          .resolveCompsDependency(targetSpec);
+
+      Queue<org.apache.hadoop.yarn.service.api.records.Component>
+          orderedCompUpgrade = new LinkedList<>();
+      resolvedComps.forEach(compName -> {
+        org.apache.hadoop.yarn.service.api.records.Component component =
+            compsNeedUpgradeByName.get(compName);
+        if (component != null ) {
+          orderedCompUpgrade.add(component);
+        }
+      });
+      event.setCompsToUpgradeInOrder(orderedCompUpgrade);
+    }
+
+    context.scheduler.getDispatcher().getEventHandler().handle(event);
+
+    if (compsNeedUpgradeList != null && !compsNeedUpgradeList.isEmpty()) {
+      if (!expressUpgrade) {
+        compsNeedUpgradeList.forEach(component -> {
           ComponentEvent needUpgradeEvent = new ComponentEvent(
               component.getName(), ComponentEventType.UPGRADE).setTargetSpec(
               component).setUpgradeVersion(event.getVersion());
           context.scheduler.getDispatcher().getEventHandler().handle(
               needUpgradeEvent);
-        } else {
-          LOG.info("The component {} has a restart "
-              + "policy that doesnt allow upgrades {} ", component.getName(),
-              component.getRestartPolicy().toString());
-        }
-      });
-    } else {
+
+        });
+      }
+    }  else if (autoFinalize) {
       // nothing to upgrade if upgrade auto finalize is requested, trigger a
       // state check.
-      if (autoFinalize) {
-        context.scheduler.getDispatcher().getEventHandler().handle(
-            new ServiceEvent(ServiceEventType.CHECK_STABLE));
-      }
+      context.scheduler.getDispatcher().getEventHandler().handle(
+          new ServiceEvent(ServiceEventType.CHECK_STABLE));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index 0801ad0..384659f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -219,7 +219,7 @@ public class ServiceScheduler extends CompositeService {
     nmClient.getClient().cleanupRunningContainersOnStop(false);
     addIfService(nmClient);
 
-    dispatcher = new AsyncDispatcher("Component  dispatcher");
+    dispatcher = createAsyncDispatcher();
     dispatcher.register(ServiceEventType.class, new ServiceEventHandler());
     dispatcher.register(ComponentEventType.class,
         new ComponentEventHandler());
@@ -253,6 +253,9 @@ public class ServiceScheduler extends CompositeService {
         YarnServiceConf.CONTAINER_RECOVERY_TIMEOUT_MS,
         YarnServiceConf.DEFAULT_CONTAINER_RECOVERY_TIMEOUT_MS,
         app.getConfiguration(), getConfig());
+
+    serviceManager = createServiceManager();
+    context.setServiceManager(serviceManager);
   }
 
   protected YarnRegistryViewForProviders createYarnRegistryOperations(
@@ -262,6 +265,14 @@ public class ServiceScheduler extends CompositeService {
         context.attemptId);
   }
 
+  protected ServiceManager createServiceManager() {
+    return new ServiceManager(context);
+  }
+
+  protected AsyncDispatcher createAsyncDispatcher() {
+    return new AsyncDispatcher("Component  dispatcher");
+  }
+
   protected NMClientAsync createNMClient() {
     return NMClientAsync.createNMClientAsync(new NMClientCallback());
   }
@@ -344,8 +355,6 @@ public class ServiceScheduler extends CompositeService {
 
     // Since AM has been started and registered, the service is in STARTED state
     app.setState(ServiceState.STARTED);
-    serviceManager = new ServiceManager(context);
-    context.setServiceManager(serviceManager);
 
     // recover components based on containers sent from RM
     recoverComponents(response);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
index b6ae38b..0b3c037 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
@@ -30,5 +30,5 @@ import org.apache.hadoop.classification.InterfaceStability;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 public enum ServiceState {
   ACCEPTED, STARTED, STABLE, STOPPED, FAILED, FLEX, UPGRADING,
-  UPGRADING_AUTO_FINALIZE;
+  UPGRADING_AUTO_FINALIZE, EXPRESS_UPGRADING;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
index 5668d9f..a27ed87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.client;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -215,48 +216,31 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return EXIT_SUCCESS;
   }
 
-  @Override
-  public int initiateUpgrade(String appName, String fileName,
-      boolean autoFinalize)
-      throws IOException, YarnException {
-    Service upgradeService = loadAppJsonFromLocalFS(fileName, appName,
-        null, null);
-    if (autoFinalize) {
-      upgradeService.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
-    } else {
-      upgradeService.setState(ServiceState.UPGRADING);
-    }
-    return initiateUpgrade(upgradeService);
-  }
-
-  public int initiateUpgrade(Service service) throws YarnException,
-      IOException {
+  private ApplicationReport upgradePrecheck(Service service)
+      throws YarnException, IOException {
     boolean upgradeEnabled = getConfig().getBoolean(
-        YARN_SERVICE_UPGRADE_ENABLED,
-        YARN_SERVICE_UPGRADE_ENABLED_DEFAULT);
+        YARN_SERVICE_UPGRADE_ENABLED, YARN_SERVICE_UPGRADE_ENABLED_DEFAULT);
     if (!upgradeEnabled) {
       throw new YarnException(ErrorStrings.SERVICE_UPGRADE_DISABLED);
     }
-    Service persistedService =
-        ServiceApiUtil.loadService(fs, service.getName());
+    Service persistedService = ServiceApiUtil.loadService(fs,
+        service.getName());
     if (!StringUtils.isEmpty(persistedService.getId())) {
-      cachedAppInfo.put(persistedService.getName(), new AppInfo(
-          ApplicationId.fromString(persistedService.getId()),
-          persistedService.getKerberosPrincipal().getPrincipalName()));
+      cachedAppInfo.put(persistedService.getName(),
+          new AppInfo(ApplicationId.fromString(persistedService.getId()),
+              persistedService.getKerberosPrincipal().getPrincipalName()));
     }
 
     if (persistedService.getVersion().equals(service.getVersion())) {
-      String message =
-          service.getName() + " is already at version " + service.getVersion()
-              + ". There is nothing to upgrade.";
+      String message = service.getName() + " is already at version "
+          + service.getVersion() + ". There is nothing to upgrade.";
       LOG.error(message);
       throw new YarnException(message);
     }
 
     Service liveService = getStatus(service.getName());
     if (!liveService.getState().equals(ServiceState.STABLE)) {
-      String message = service.getName() + " is at " +
-          liveService.getState()
+      String message = service.getName() + " is at " + liveService.getState()
           + " state and upgrade can only be initiated when service is STABLE.";
       LOG.error(message);
       throw new YarnException(message);
@@ -266,11 +250,67 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
     ServiceApiUtil.createDirAndPersistApp(fs, serviceUpgradeDir, service);
 
-    ApplicationReport appReport =
-        yarnClient.getApplicationReport(getAppId(service.getName()));
+    ApplicationReport appReport = yarnClient
+        .getApplicationReport(getAppId(service.getName()));
     if (StringUtils.isEmpty(appReport.getHost())) {
       throw new YarnException(service.getName() + " AM hostname is empty");
     }
+    return appReport;
+  }
+
+  @Override
+  public int actionUpgradeExpress(String appName, File path)
+      throws IOException, YarnException {
+    Service service =
+        loadAppJsonFromLocalFS(path.getAbsolutePath(), appName, null, null);
+    service.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
+    actionUpgradeExpress(service);
+    return EXIT_SUCCESS;
+  }
+
+  public int actionUpgradeExpress(Service service) throws YarnException,
+      IOException {
+    ApplicationReport appReport = upgradePrecheck(service);
+    ClientAMProtocol proxy = createAMProxy(service.getName(), appReport);
+    UpgradeServiceRequestProto.Builder requestBuilder =
+        UpgradeServiceRequestProto.newBuilder();
+    requestBuilder.setVersion(service.getVersion());
+    if (service.getState().equals(ServiceState.UPGRADING_AUTO_FINALIZE)) {
+      requestBuilder.setAutoFinalize(true);
+    }
+    if (service.getState().equals(ServiceState.EXPRESS_UPGRADING)) {
+      requestBuilder.setExpressUpgrade(true);
+      requestBuilder.setAutoFinalize(true);
+    }
+    UpgradeServiceResponseProto responseProto = proxy.upgrade(
+        requestBuilder.build());
+    if (responseProto.hasError()) {
+      LOG.error("Service {} express upgrade to version {} failed because {}",
+          service.getName(), service.getVersion(), responseProto.getError());
+      throw new YarnException("Failed to express upgrade service " +
+          service.getName() + " to version " + service.getVersion() +
+          " because " + responseProto.getError());
+    }
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public int initiateUpgrade(String appName, String fileName,
+      boolean autoFinalize)
+      throws IOException, YarnException {
+    Service upgradeService = loadAppJsonFromLocalFS(fileName, appName,
+        null, null);
+    if (autoFinalize) {
+      upgradeService.setState(ServiceState.UPGRADING_AUTO_FINALIZE);
+    } else {
+      upgradeService.setState(ServiceState.UPGRADING);
+    }
+    return initiateUpgrade(upgradeService);
+  }
+
+  public int initiateUpgrade(Service service) throws YarnException,
+      IOException {
+    ApplicationReport appReport = upgradePrecheck(service);
     ClientAMProtocol proxy = createAMProxy(service.getName(), appReport);
 
     UpgradeServiceRequestProto.Builder requestBuilder =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index 41a2fcd..acf3404 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.service.component;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import static org.apache.hadoop.yarn.service.api.records.Component
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.service.ServiceEventType;
 import org.apache.hadoop.yarn.service.api.records.ContainerState;
 import org.apache.hadoop.yarn.service.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
 import org.apache.hadoop.yarn.service.ContainerFailureTracker;
 import org.apache.hadoop.yarn.service.ServiceContext;
@@ -546,13 +548,21 @@ public class Component implements EventHandler<ComponentEvent> {
     @Override
     public void transition(Component component, ComponentEvent event) {
       component.upgradeInProgress.set(true);
+      component.upgradeEvent = event;
       component.componentSpec.setState(org.apache.hadoop.yarn.service.api.
           records.ComponentState.NEEDS_UPGRADE);
       component.numContainersThatNeedUpgrade.set(
           component.componentSpec.getNumberOfContainers());
-      component.componentSpec.getContainers().forEach(container ->
-          container.setState(ContainerState.NEEDS_UPGRADE));
-      component.upgradeEvent = event;
+      component.componentSpec.getContainers().forEach(container -> {
+        container.setState(ContainerState.NEEDS_UPGRADE);
+        if (event.isExpressUpgrade()) {
+          ComponentInstanceEvent upgradeEvent = new ComponentInstanceEvent(
+              ContainerId.fromString(container.getId()),
+                  ComponentInstanceEventType.UPGRADE);
+          LOG.info("Upgrade container {}", container.getId());
+          component.dispatcher.getEventHandler().handle(upgradeEvent);
+        }
+      });
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java
index 84caa77..643961d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java
@@ -35,6 +35,7 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
   private ContainerId containerId;
   private org.apache.hadoop.yarn.service.api.records.Component targetSpec;
   private String upgradeVersion;
+  private boolean expressUpgrade;
 
   public ContainerId getContainerId() {
     return containerId;
@@ -113,4 +114,13 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
     this.upgradeVersion = upgradeVersion;
     return this;
   }
+
+  public boolean isExpressUpgrade() {
+    return expressUpgrade;
+  }
+
+  public ComponentEvent setExpressUpgrade(boolean expressUpgrade) {
+    this.expressUpgrade = expressUpgrade;
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
index 11a6caa..ed5e68e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
@@ -380,6 +380,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
+      if (!compInstance.containerSpec.getState().equals(
+          ContainerState.NEEDS_UPGRADE)) {
+        //nothing to upgrade. this may happen with express upgrade.
+        return;
+      }
       compInstance.containerSpec.setState(ContainerState.UPGRADING);
       compInstance.component.decContainersReady(false);
       ComponentEvent upgradeEvent = compInstance.component.getUpgradeEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index 9219569..b588e88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -638,6 +638,32 @@ public class ServiceApiUtil {
     return containerNeedUpgrade;
   }
 
+  /**
+   * Validates the components that are requested are stable for upgrade.
+   * It returns the instances of the components which are in ready state.
+   */
+  public static List<Container> validateAndResolveCompsStable(
+      Service liveService, Collection<String> compNames) throws YarnException {
+    Preconditions.checkNotNull(compNames);
+    HashSet<String> requestedComps = Sets.newHashSet(compNames);
+    List<Container> containerNeedUpgrade = new ArrayList<>();
+    for (Component liveComp : liveService.getComponents()) {
+      if (requestedComps.contains(liveComp.getName())) {
+        if (!liveComp.getState().equals(ComponentState.STABLE)) {
+          // Nothing to upgrade
+          throw new YarnException(String.format(
+              ERROR_COMP_DOES_NOT_NEED_UPGRADE, liveComp.getName()));
+        }
+        liveComp.getContainers().forEach(liveContainer -> {
+          if (liveContainer.getState().equals(ContainerState.READY)) {
+            containerNeedUpgrade.add(liveContainer);
+          }
+        });
+      }
+    }
+    return containerNeedUpgrade;
+  }
+
   private static String parseComponentName(String componentInstanceName)
       throws YarnException {
     int idx = componentInstanceName.lastIndexOf('-');
@@ -651,4 +677,22 @@ public class ServiceApiUtil {
   public static String $(String s) {
     return "${" + s +"}";
   }
+
+  public static List<String> resolveCompsDependency(Service service) {
+    List<String> components = new ArrayList<String>();
+    for (Component component : service.getComponents()) {
+      int depSize = component.getDependencies().size();
+      if (!components.contains(component.getName())) {
+        components.add(component.getName());
+      }
+      if (depSize != 0) {
+        for (String depComp : component.getDependencies()) {
+          if (!components.contains(depComp)) {
+            components.add(0, depComp);
+          }
+        }
+      }
+    }
+    return components;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
index 6166ded..169f765 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
@@ -66,6 +66,7 @@ message StopResponseProto {
 message UpgradeServiceRequestProto {
   optional string version = 1;
   optional bool autoFinalize = 2;
+  optional bool expressUpgrade = 3;
 }
 
 message UpgradeServiceResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
deleted file mode 100644
index c2a80e7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceApiUtil.java
+++ /dev/null
@@ -1,653 +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.yarn.service;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.registry.client.api.RegistryConstants;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.service.api.records.Artifact;
-import org.apache.hadoop.yarn.service.api.records.Component;
-import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
-import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
-import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
-import org.apache.hadoop.yarn.service.api.records.PlacementScope;
-import org.apache.hadoop.yarn.service.api.records.PlacementType;
-import org.apache.hadoop.yarn.service.api.records.Resource;
-import org.apache.hadoop.yarn.service.api.records.Service;
-import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
-import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
-import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME;
-import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test for ServiceApiUtil helper methods.
- */
-public class TestServiceApiUtil {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestServiceApiUtil.class);
-  private static final String EXCEPTION_PREFIX = "Should have thrown " +
-      "exception: ";
-  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown " +
-      "exception: ";
-
-  private static final String LEN_64_STR =
-      "abcdefghijklmnopqrstuvwxyz0123456789abcdefghijklmnopqrstuvwxyz01";
-
-  private static final YarnConfiguration CONF_DEFAULT_DNS = new
-      YarnConfiguration();
-  private static final YarnConfiguration CONF_DNS_ENABLED = new
-      YarnConfiguration();
-
-  @BeforeClass
-  public static void init() {
-    CONF_DNS_ENABLED.setBoolean(RegistryConstants.KEY_DNS_ENABLED, true);
-  }
-
-  @Test(timeout = 90000)
-  public void testResourceValidation() throws Exception {
-    assertEquals(RegistryConstants.MAX_FQDN_LABEL_LENGTH + 1, LEN_64_STR
-        .length());
-
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    Service app = new Service();
-
-    // no name
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
-    }
-
-    app.setName("test");
-    // no version
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + " service with no version");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(ERROR_APPLICATION_VERSION_INVALID,
-          app.getName()), e.getMessage());
-    }
-
-    app.setVersion("v1");
-    // bad format name
-    String[] badNames = {"4finance", "Finance", "finance@home", LEN_64_STR};
-    for (String badName : badNames) {
-      app.setName(badName);
-      try {
-        ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-        Assert.fail(EXCEPTION_PREFIX + "service with bad name " + badName);
-      } catch (IllegalArgumentException e) {
-
-      }
-    }
-
-    // launch command not specified
-    app.setName(LEN_64_STR);
-    Component comp = new Component().name("comp1");
-    app.addComponent(comp);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
-      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
-    } catch (IllegalArgumentException e) {
-      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
-          e.getMessage());
-    }
-
-    // launch command not specified
-    app.setName(LEN_64_STR.substring(0, RegistryConstants
-        .MAX_FQDN_LABEL_LENGTH));
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
-    } catch (IllegalArgumentException e) {
-      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
-          e.getMessage());
-    }
-
-    // memory not specified
-    comp.setLaunchCommand("sleep 1");
-    Resource res = new Resource();
-    app.setResource(res);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no memory");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID,
-          comp.getName()), e.getMessage());
-    }
-
-    // invalid no of cpus
-    res.setMemory("100mb");
-    res.setCpus(-2);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(
-          EXCEPTION_PREFIX + "service with invalid no of cpus");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE,
-          comp.getName()), e.getMessage());
-    }
-
-    // number of containers not specified
-    res.setCpus(2);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no container count");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage()
-          .contains(ERROR_CONTAINERS_COUNT_INVALID));
-    }
-
-    // specifying profile along with cpus/memory raises exception
-    res.setProfile("hbase_finance_large");
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX
-          + "service with resource profile along with cpus/memory");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(RestApiErrorMessages
-              .ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
-          comp.getName()),
-          e.getMessage());
-    }
-
-    // currently resource profile alone is not supported.
-    // TODO: remove the next test once resource profile alone is supported.
-    res.setCpus(null);
-    res.setMemory(null);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with resource profile only");
-    } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
-          e.getMessage());
-    }
-
-    // unset profile here and add cpus/memory back
-    res.setProfile(null);
-    res.setCpus(2);
-    res.setMemory("2gb");
-
-    // null number of containers
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "null number of containers");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith(ERROR_CONTAINERS_COUNT_INVALID));
-    }
-  }
-
-  @Test
-  public void testArtifacts() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    Service app = new Service();
-    app.setName("service1");
-    app.setVersion("v1");
-    Resource res = new Resource();
-    app.setResource(res);
-    res.setMemory("512M");
-
-    // no artifact id fails with default type
-    Artifact artifact = new Artifact();
-    app.setArtifact(artifact);
-    String compName = "comp1";
-    Component comp = ServiceTestUtils.createComponent(compName);
-
-    app.setComponents(Collections.singletonList(comp));
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
-          e.getMessage());
-    }
-
-    // no artifact id fails with SERVICE type
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
-    } catch (IllegalArgumentException e) {
-      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
-    }
-
-    // no artifact id fails with TARBALL type
-    artifact.setType(Artifact.TypeEnum.TARBALL);
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
-          e.getMessage());
-    }
-
-    // everything valid here
-    artifact.setType(Artifact.TypeEnum.DOCKER);
-    artifact.setId("docker.io/centos:centos7");
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      LOG.error("service attributes specified should be valid here", e);
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
-  }
-
-  private static Resource createValidResource() {
-    Resource res = new Resource();
-    res.setMemory("512M");
-    return res;
-  }
-
-  private static Component createValidComponent(String compName) {
-    Component comp = new Component();
-    comp.setName(compName);
-    comp.setResource(createValidResource());
-    comp.setNumberOfContainers(1L);
-    comp.setLaunchCommand("sleep 1");
-    return comp;
-  }
-
-  private static Service createValidApplication(String compName) {
-    Service app = new Service();
-    app.setName("name");
-    app.setVersion("v1");
-    app.setResource(createValidResource());
-    if (compName != null) {
-      app.addComponent(createValidComponent(compName));
-    }
-    return app;
-  }
-
-  @Test
-  public void testExternalApplication() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication(null);
-
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.setArtifact(artifact);
-    app.addComponent(ServiceTestUtils.createComponent("comp2"));
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    assertNotNull(app.getComponent("comp2"));
-  }
-
-  @Test
-  public void testDuplicateComponents() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-
-    String compName = "comp1";
-    Service app = createValidApplication(compName);
-    app.addComponent(createValidComponent(compName));
-
-    // duplicate component name fails
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with component collision");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Component name collision: " + compName, e.getMessage());
-    }
-  }
-
-  @Test
-  public void testComponentNameSameAsServiceName() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = new Service();
-    app.setName("test");
-    app.setVersion("v1");
-    app.addComponent(createValidComponent("test"));
-
-    //component name same as service name
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "component name matches service name");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Component name test must not be same as service name test",
-          e.getMessage());
-    }
-  }
-
-  @Test
-  public void testExternalDuplicateComponent() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication("comp1");
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.getComponent("comp1").setArtifact(artifact);
-
-    // duplicate component name okay in the case of SERVICE component
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testExternalComponent() throws IOException {
-    Service ext = createValidApplication("comp1");
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
-
-    Service app = createValidApplication("comp2");
-    Artifact artifact = new Artifact();
-    artifact.setType(Artifact.TypeEnum.SERVICE);
-    artifact.setId("id");
-    app.setArtifact(artifact);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    // artifact ID not inherited from global
-    assertNotNull(app.getComponent("comp2"));
-
-    // set SERVICE artifact id on component
-    app.getComponent("comp2").setArtifact(artifact);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    assertEquals(1, app.getComponents().size());
-    // original component replaced by external component
-    assertNotNull(app.getComponent("comp1"));
-  }
-
-  public static void verifyDependencySorting(List<Component> components,
-      Component... expectedSorting) {
-    Collection<Component> actualSorting = ServiceApiUtil.sortByDependencies(
-        components);
-    assertEquals(expectedSorting.length, actualSorting.size());
-    int i = 0;
-    for (Component component : actualSorting) {
-      assertEquals(expectedSorting[i++], component);
-    }
-  }
-
-  @Test
-  public void testDependencySorting() throws IOException {
-    Component a = ServiceTestUtils.createComponent("a");
-    Component b = ServiceTestUtils.createComponent("b");
-    Component c = ServiceTestUtils.createComponent("c");
-    Component d =
-        ServiceTestUtils.createComponent("d").dependencies(Arrays.asList("c"));
-    Component e = ServiceTestUtils.createComponent("e")
-        .dependencies(Arrays.asList("b", "d"));
-
-    verifyDependencySorting(Arrays.asList(a, b, c), a, b, c);
-    verifyDependencySorting(Arrays.asList(c, a, b), c, a, b);
-    verifyDependencySorting(Arrays.asList(a, b, c, d, e), a, b, c, d, e);
-    verifyDependencySorting(Arrays.asList(e, d, c, b, a), c, b, a, d, e);
-
-    c.setDependencies(Arrays.asList("e"));
-    try {
-      verifyDependencySorting(Arrays.asList(a, b, c, d, e));
-      Assert.fail(EXCEPTION_PREFIX + "components with dependency cycle");
-    } catch (IllegalArgumentException ex) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_DEPENDENCY_CYCLE, Arrays.asList(c, d,
-              e)), ex.getMessage());
-    }
-
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service service = createValidApplication(null);
-    service.setComponents(Arrays.asList(c, d, e));
-    try {
-      ServiceApiUtil.validateAndResolveService(service, sfs,
-          CONF_DEFAULT_DNS);
-      Assert.fail(EXCEPTION_PREFIX + "components with bad dependencies");
-    } catch (IllegalArgumentException ex) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_DEPENDENCY_INVALID, "b", "e"), ex
-          .getMessage());
-    }
-  }
-
-  @Test
-  public void testInvalidComponent() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    testComponent(sfs);
-  }
-
-  @Test
-  public void testValidateCompName() {
-    String[] invalidNames = {
-        "EXAMPLE", // UPPER case not allowed
-        "example_app" // underscore not allowed.
-    };
-    for (String name : invalidNames) {
-      try {
-        ServiceApiUtil.validateNameFormat(name, new Configuration());
-        Assert.fail();
-      } catch (IllegalArgumentException ex) {
-        ex.printStackTrace();
-      }
-    }
-  }
-
-  private static void testComponent(SliderFileSystem sfs)
-      throws IOException {
-    int maxLen = RegistryConstants.MAX_FQDN_LABEL_LENGTH;
-    assertEquals(19, Long.toString(Long.MAX_VALUE).length());
-    maxLen = maxLen - Long.toString(Long.MAX_VALUE).length();
-
-    String compName = LEN_64_STR.substring(0, maxLen + 1);
-    Service app = createValidApplication(null);
-    app.addComponent(createValidComponent(compName));
-
-    // invalid component name fails if dns is enabled
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid component name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(RestApiErrorMessages
-          .ERROR_COMPONENT_NAME_INVALID, maxLen, compName), e.getMessage());
-    }
-
-    // does not fail if dns is disabled
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-
-    compName = LEN_64_STR.substring(0, maxLen);
-    app = createValidApplication(null);
-    app.addComponent(createValidComponent(compName));
-
-    // does not fail
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testPlacementPolicy() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = createValidApplication("comp-a");
-    Component comp = app.getComponents().get(0);
-    PlacementPolicy pp = new PlacementPolicy();
-    PlacementConstraint pc = new PlacementConstraint();
-    pc.setName("CA1");
-    pp.setConstraints(Collections.singletonList(pc));
-    comp.setPlacementPolicy(pp);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no type");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set the type
-    pc.setType(PlacementType.ANTI_AFFINITY);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no scope");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set the scope
-    pc.setScope(PlacementScope.NODE);
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with no tag(s)");
-    } catch (IllegalArgumentException e) {
-      assertEquals(String.format(
-          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
-          "CA1 ", "comp-a"), e.getMessage());
-    }
-
-    // Set a target tag - but an invalid one
-    pc.setTargetTags(Collections.singletonList("comp-invalid"));
-
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-      Assert.fail(EXCEPTION_PREFIX + "constraint with invalid tag name");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(
-              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_TAG_NAME_NOT_SAME,
-              "comp-invalid", "comp-a", "comp-a", "comp-a"),
-          e.getMessage());
-    }
-
-    // Set valid target tags now
-    pc.setTargetTags(Collections.singletonList("comp-a"));
-
-    // Finally it should succeed
-    try {
-      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testKerberosPrincipal() throws IOException {
-    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
-    Service app = createValidApplication("comp-a");
-    KerberosPrincipal kp = new KerberosPrincipal();
-    kp.setKeytab("/some/path");
-    kp.setPrincipalName("user/_HOST@domain.com");
-    app.setKerberosPrincipal(kp);
-
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab URI scheme");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(RestApiErrorMessages.ERROR_KEYTAB_URI_SCHEME_INVALID,
-              kp.getKeytab()),
-          e.getMessage());
-    }
-
-    kp.setKeytab("/ blank / in / paths");
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab");
-    } catch (IllegalArgumentException e) {
-      // strip out the %s at the end of the RestApiErrorMessages string constant
-      assertTrue(e.getMessage().contains(
-          RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.substring(0,
-              RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.length() - 2)));
-    }
-
-    kp.setKeytab("file:///tmp/a.keytab");
-    // now it should succeed
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-
-  @Test
-  public void testKerberosPrincipalNameFormat() throws IOException {
-    Service app = createValidApplication("comp-a");
-    KerberosPrincipal kp = new KerberosPrincipal();
-    kp.setPrincipalName("user@domain.com");
-    app.setKerberosPrincipal(kp);
-
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-      Assert.fail(EXCEPTION_PREFIX + "service with invalid principal name format.");
-    } catch (IllegalArgumentException e) {
-      assertEquals(
-          String.format(RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
-              kp.getPrincipalName()),
-          e.getMessage());
-    }
-
-    kp.setPrincipalName("user/_HOST@domain.com");
-    try {
-      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
-    } catch (IllegalArgumentException e) {
-      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.java
index fc509f1..a37cabe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceManager.java
@@ -19,23 +19,26 @@
 package org.apache.hadoop.yarn.service;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
 import org.apache.hadoop.yarn.service.api.records.ComponentState;
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
 import org.apache.hadoop.yarn.service.exceptions.SliderException;
-import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Map;
-
-import static org.mockito.Mockito.mock;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
 
 /**
  * Tests for {@link ServiceManager}.
@@ -46,117 +49,120 @@ public class TestServiceManager {
   public ServiceTestUtils.ServiceFSWatcher rule =
       new ServiceTestUtils.ServiceFSWatcher();
 
-  @Test
-  public void testUpgrade() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager("testUpgrade");
-    upgrade(serviceManager, "v2", false, false);
+  @Test (timeout = TIMEOUT)
+  public void testUpgrade() throws Exception {
+    ServiceContext context = createServiceContext("testUpgrade");
+    initUpgrade(context, "v2", false, false, false);
     Assert.assertEquals("service not upgraded", ServiceState.UPGRADING,
-        serviceManager.getServiceSpec().getState());
+        context.getServiceManager().getServiceSpec().getState());
   }
 
-  @Test
+  @Test (timeout = TIMEOUT)
   public void testRestartNothingToUpgrade()
-      throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
+      throws Exception {
+    ServiceContext context = createServiceContext(
         "testRestartNothingToUpgrade");
-    upgrade(serviceManager, "v2", false, false);
-
-    //make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp -> {
-      comp.setState(ComponentState.STABLE);
-    });
-    serviceManager.handle(new ServiceEvent(ServiceEventType.START));
+    initUpgrade(context, "v2", false, false, false);
+    ServiceManager manager = context.getServiceManager();
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
+
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service not re-started", ServiceState.STABLE,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
   }
 
-  @Test
-  public void testAutoFinalizeNothingToUpgrade() throws IOException,
-      SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
+  @Test(timeout = TIMEOUT)
+  public void testAutoFinalizeNothingToUpgrade() throws Exception {
+    ServiceContext context = createServiceContext(
         "testAutoFinalizeNothingToUpgrade");
-    upgrade(serviceManager, "v2", false, true);
-
-    //make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp ->
-        comp.setState(ComponentState.STABLE));
-    serviceManager.handle(new ServiceEvent(ServiceEventType.CHECK_STABLE));
+    initUpgrade(context, "v2", false, true, false);
+    ServiceManager manager = context.getServiceManager();
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
+
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service stable", ServiceState.STABLE,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
   }
 
-  @Test
+  @Test(timeout = TIMEOUT)
   public void testRestartWithPendingUpgrade()
-      throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager("testRestart");
-    upgrade(serviceManager, "v2", true, false);
-    serviceManager.handle(new ServiceEvent(ServiceEventType.START));
+      throws Exception {
+    ServiceContext context = createServiceContext("testRestart");
+    initUpgrade(context, "v2", true, false, false);
+    ServiceManager manager = context.getServiceManager();
+
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    context.scheduler.getDispatcher().stop();
     Assert.assertEquals("service should still be upgrading",
-        ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
+        ServiceState.UPGRADING, manager.getServiceSpec().getState());
   }
 
-  @Test
-  public void testCheckState() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testCheckState");
-    upgrade(serviceManager, "v2", true, false);
+  @Test(timeout = TIMEOUT)
+  public void testFinalize() throws Exception {
+    ServiceContext context = createServiceContext("testCheckState");
+    initUpgrade(context, "v2", true, false, false);
+    ServiceManager manager = context.getServiceManager();
     Assert.assertEquals("service not upgrading", ServiceState.UPGRADING,
-        serviceManager.getServiceSpec().getState());
+        manager.getServiceSpec().getState());
 
-    // make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp -> {
-      comp.setState(ComponentState.STABLE);
-    });
-    ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
-    serviceManager.handle(checkStable);
-    Assert.assertEquals("service should still be upgrading",
-        ServiceState.UPGRADING, serviceManager.getServiceSpec().getState());
+    //make components stable by upgrading all instances
+    upgradeAllInstances(context);
 
     // finalize service
-    ServiceEvent restart = new ServiceEvent(ServiceEventType.START);
-    serviceManager.handle(restart);
-    Assert.assertEquals("service not stable",
-        ServiceState.STABLE, serviceManager.getServiceSpec().getState());
+    context.scheduler.getDispatcher().getEventHandler().handle(
+        new ServiceEvent(ServiceEventType.START));
+    GenericTestUtils.waitFor(()->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
+    Assert.assertEquals("service not re-started", ServiceState.STABLE,
+        manager.getServiceSpec().getState());
 
-    validateUpgradeFinalization(serviceManager.getName(), "v2");
+    validateUpgradeFinalization(manager.getName(), "v2");
   }
 
-  @Test
-  public void testCheckStateAutoFinalize() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testCheckState");
-    serviceManager.getServiceSpec().setState(
+  @Test(timeout = TIMEOUT)
+  public void testAutoFinalize() throws Exception {
+    ServiceContext context = createServiceContext("testCheckStateAutoFinalize");
+    ServiceManager manager = context.getServiceManager();
+    manager.getServiceSpec().setState(
         ServiceState.UPGRADING_AUTO_FINALIZE);
-    upgrade(serviceManager, "v2", true, true);
-    Assert.assertEquals("service not upgrading",
-        ServiceState.UPGRADING_AUTO_FINALIZE,
-        serviceManager.getServiceSpec().getState());
+    initUpgrade(context, "v2", true, true, false);
 
     // make components stable
-    serviceManager.getServiceSpec().getComponents().forEach(comp ->
-        comp.setState(ComponentState.STABLE));
-    ServiceEvent checkStable = new ServiceEvent(ServiceEventType.CHECK_STABLE);
-    serviceManager.handle(checkStable);
+    upgradeAllInstances(context);
+
+    GenericTestUtils.waitFor(() ->
+        context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
     Assert.assertEquals("service not stable",
-        ServiceState.STABLE, serviceManager.getServiceSpec().getState());
+        ServiceState.STABLE, manager.getServiceSpec().getState());
 
-    validateUpgradeFinalization(serviceManager.getName(), "v2");
+    validateUpgradeFinalization(manager.getName(), "v2");
   }
 
   @Test
-  public void testInvalidUpgrade() throws IOException, SliderException {
-    ServiceManager serviceManager = createTestServiceManager(
-        "testInvalidUpgrade");
-    serviceManager.getServiceSpec().setState(
+  public void testInvalidUpgrade() throws Exception {
+    ServiceContext serviceContext = createServiceContext("testInvalidUpgrade");
+    ServiceManager manager = serviceContext.getServiceManager();
+    manager.getServiceSpec().setState(
         ServiceState.UPGRADING_AUTO_FINALIZE);
     Service upgradedDef = ServiceTestUtils.createExampleApplication();
-    upgradedDef.setName(serviceManager.getName());
+    upgradedDef.setName(manager.getName());
     upgradedDef.setVersion("v2");
     upgradedDef.setLifetime(2L);
     writeUpgradedDef(upgradedDef);
 
     try {
-      serviceManager.processUpgradeRequest("v2", true);
+      manager.processUpgradeRequest("v2", true, false);
     } catch (Exception ex) {
       Assert.assertTrue(ex instanceof UnsupportedOperationException);
       return;
@@ -164,6 +170,32 @@ public class TestServiceManager {
     Assert.fail();
   }
 
+  @Test(timeout = TIMEOUT)
+  public void testExpressUpgrade() throws Exception {
+    ServiceContext context = createServiceContext("testExpressUpgrade");
+    ServiceManager manager = context.getServiceManager();
+    manager.getServiceSpec().setState(
+        ServiceState.EXPRESS_UPGRADING);
+    initUpgrade(context, "v2", true, true, true);
+
+    List<String> comps = ServiceApiUtil.resolveCompsDependency(context.service);
+    // wait till instances of first component are in upgrade
+    String comp1 = comps.get(0);
+    upgradeInstancesOf(context, comp1);
+
+    // wait till instances of second component are in upgrade
+    String comp2 = comps.get(1);
+    upgradeInstancesOf(context, comp2);
+
+    GenericTestUtils.waitFor(() ->
+            context.service.getState().equals(ServiceState.STABLE),
+        CHECK_EVERY_MILLIS, TIMEOUT);
+
+    Assert.assertEquals("service not stable",
+        ServiceState.STABLE, manager.getServiceSpec().getState());
+    validateUpgradeFinalization(manager.getName(), "v2");
+  }
+
   private void validateUpgradeFinalization(String serviceName,
       String expectedVersion) throws IOException {
     Service savedSpec = ServiceApiUtil.loadService(rule.getFs(), serviceName);
@@ -172,15 +204,16 @@ public class TestServiceManager {
     Assert.assertNotNull("app id not present", savedSpec.getId());
     Assert.assertEquals("state not stable", ServiceState.STABLE,
         savedSpec.getState());
-    savedSpec.getComponents().forEach(compSpec -> {
-      Assert.assertEquals("comp not stable", ComponentState.STABLE,
-          compSpec.getState());
-    });
+    savedSpec.getComponents().forEach(compSpec ->
+        Assert.assertEquals("comp not stable", ComponentState.STABLE,
+        compSpec.getState()));
   }
 
-  private void upgrade(ServiceManager serviceManager, String version,
-      boolean upgradeArtifact, boolean autoFinalize)
-      throws IOException, SliderException {
+  private void initUpgrade(ServiceContext context, String version,
+      boolean upgradeArtifact, boolean autoFinalize, boolean expressUpgrade)
+      throws IOException, SliderException, TimeoutException,
+      InterruptedException {
+    ServiceManager serviceManager = context.getServiceManager();
     Service upgradedDef = ServiceTestUtils.createExampleApplication();
     upgradedDef.setName(serviceManager.getName());
     upgradedDef.setVersion(version);
@@ -191,39 +224,81 @@ public class TestServiceManager {
       });
     }
     writeUpgradedDef(upgradedDef);
-    serviceManager.processUpgradeRequest(version, autoFinalize);
+    serviceManager.processUpgradeRequest(version, autoFinalize, expressUpgrade);
     ServiceEvent upgradeEvent = new ServiceEvent(ServiceEventType.UPGRADE);
-    upgradeEvent.setVersion(version);
-    if (autoFinalize) {
-      upgradeEvent.setAutoFinalize(true);
-    }
-    serviceManager.handle(upgradeEvent);
+    upgradeEvent.setVersion(version).setExpressUpgrade(expressUpgrade)
+        .setAutoFinalize(autoFinalize);
+
+    GenericTestUtils.waitFor(()-> {
+      ServiceState serviceState = context.service.getState();
+      if (serviceState.equals(ServiceState.UPGRADING) ||
+          serviceState.equals(ServiceState.UPGRADING_AUTO_FINALIZE) ||
+          serviceState.equals(ServiceState.EXPRESS_UPGRADING)) {
+        return true;
+      }
+      return false;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
+  }
+
+  private void upgradeAllInstances(ServiceContext context) throws
+      TimeoutException, InterruptedException {
+    // upgrade the instances
+    context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
+          ComponentInstanceEventType.UPGRADE);
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
+    }));
+
+    // become ready
+    context.scheduler.getLiveInstances().forEach(((containerId, instance) -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(containerId,
+          ComponentInstanceEventType.BECOME_READY);
+
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
+    }));
+    GenericTestUtils.waitFor(()-> {
+      for (ComponentInstance instance:
+          context.scheduler.getLiveInstances().values()) {
+        if (!instance.getContainerState().equals(ContainerState.READY)) {
+          return false;
+        }
+      }
+      return true;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
   }
 
-  private ServiceManager createTestServiceManager(String name)
-      throws IOException {
-    ServiceContext context = new ServiceContext();
-    context.service = createBaseDef(name);
-    context.fs = rule.getFs();
-
-    context.scheduler = new ServiceScheduler(context) {
-      @Override
-      protected YarnRegistryViewForProviders createYarnRegistryOperations(
-          ServiceContext context, RegistryOperations registryClient) {
-        return mock(YarnRegistryViewForProviders.class);
+  private void upgradeInstancesOf(ServiceContext context, String compName)
+      throws TimeoutException, InterruptedException {
+    Collection<ComponentInstance> compInstances = context.scheduler
+        .getAllComponents().get(compName).getAllComponentInstances();
+    GenericTestUtils.waitFor(() -> {
+      for (ComponentInstance instance : compInstances) {
+        if (!instance.getContainerState().equals(ContainerState.UPGRADING)) {
+          return false;
+        }
       }
-    };
+      return true;
+    }, CHECK_EVERY_MILLIS, TIMEOUT);
 
-    context.scheduler.init(rule.getConf());
+    // instances of comp1 get upgraded and become ready event is triggered
+    // become ready
+    compInstances.forEach(instance -> {
+      ComponentInstanceEvent event = new ComponentInstanceEvent(
+          instance.getContainer().getId(),
+          ComponentInstanceEventType.BECOME_READY);
 
-    Map<String, org.apache.hadoop.yarn.service.component.Component>
-        componentState = context.scheduler.getAllComponents();
-    context.service.getComponents().forEach(component -> {
-      componentState.put(component.getName(),
-          new org.apache.hadoop.yarn.service.component.Component(component,
-              1L, context));
+      context.scheduler.getDispatcher().getEventHandler().handle(event);
     });
-    return new ServiceManager(context);
+  }
+
+  private ServiceContext createServiceContext(String name)
+      throws Exception {
+    Service service  = createBaseDef(name);
+    ServiceContext context = new MockRunningServiceContext(rule,
+        service);
+    context.scheduler.getDispatcher().setDrainEventsOnStop();
+    context.scheduler.getDispatcher().start();
+    return context;
   }
 
   public static Service createBaseDef(String name) {
@@ -257,4 +332,6 @@ public class TestServiceManager {
         upgradedDef);
   }
 
+  private static final int TIMEOUT = 200000;
+  private static final int CHECK_EVERY_MILLIS = 100;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
index 8b13b24..216d88f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
@@ -415,6 +415,41 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     client.actionDestroy(service.getName());
   }
 
+  @Test(timeout = 200000)
+  public void testExpressUpgrade() throws Exception {
+    setupInternal(NUM_NMS);
+    getConf().setBoolean(YARN_SERVICE_UPGRADE_ENABLED, true);
+    ServiceClient client = createClient(getConf());
+
+    Service service = createExampleApplication();
+    client.actionCreate(service);
+    waitForServiceToBeStable(client, service);
+
+    // upgrade the service
+    Component component = service.getComponents().iterator().next();
+    service.setState(ServiceState.EXPRESS_UPGRADING);
+    service.setVersion("v2");
+    component.getConfiguration().getEnv().put("key1", "val1");
+    Component component2 = service.getComponent("compb");
+    component2.getConfiguration().getEnv().put("key2", "val2");
+    client.actionUpgradeExpress(service);
+
+    // wait for upgrade to complete
+    waitForServiceToBeStable(client, service);
+    Service active = client.getStatus(service.getName());
+    Assert.assertEquals("component not stable", ComponentState.STABLE,
+        active.getComponent(component.getName()).getState());
+    Assert.assertEquals("compa does not have new env", "val1",
+        active.getComponent(component.getName()).getConfiguration()
+            .getEnv("key1"));
+    Assert.assertEquals("compb does not have new env", "val2",
+        active.getComponent(component2.getName()).getConfiguration()
+            .getEnv("key2"));
+    LOG.info("Stop/destroy service {}", service);
+    client.actionStop(service.getName(), true);
+    client.actionDestroy(service.getName());
+  }
+
   // Test to verify ANTI_AFFINITY placement policy
   // 1. Start mini cluster with 3 NMs and scheduler placement-constraint handler
   // 2. Create an example service with 3 containers


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


[42/50] [abbrv] hadoop git commit: YARN-8632. Threads in SLS quit without logging exception. Contributed by Xianghao Lu.

Posted by sh...@apache.org.
YARN-8632. Threads in SLS quit without logging exception. Contributed by Xianghao Lu.


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

Branch: refs/heads/HDFS-12943
Commit: 8563fd67befeb0026a95ae268395baf04cd3c22f
Parents: 138b0c1
Author: Yufei Gu <yu...@apple.com>
Authored: Fri Aug 24 11:23:17 2018 -0700
Committer: Yufei Gu <yu...@apple.com>
Committed: Fri Aug 24 11:23:17 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8563fd67/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
index b8bc8be..2957d23 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java
@@ -32,7 +32,6 @@ import java.util.List;
 import java.util.SortedMap;
 import java.util.Locale;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.Lock;
@@ -48,6 +47,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.concurrent.HadoopScheduledThreadPoolExecutor;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
@@ -169,7 +169,7 @@ public abstract class SchedulerMetrics {
     web.start();
 
     // a thread to update histogram timer
-    pool = new ScheduledThreadPoolExecutor(2);
+    pool = new HadoopScheduledThreadPoolExecutor(2);
     pool.scheduleAtFixedRate(new HistogramsRunnable(), 0, 1000,
         TimeUnit.MILLISECONDS);
 
@@ -518,7 +518,8 @@ public abstract class SchedulerMetrics {
 
     @Override
     public void run() {
-      if(running) {
+      SchedulerWrapper wrapper = (SchedulerWrapper) scheduler;
+      if(running && wrapper.getTracker().getQueueSet() != null) {
         // all WebApp to get real tracking json
         String trackingMetrics = web.generateRealTimeTrackingMetrics();
         // output


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


[27/50] [abbrv] hadoop git commit: YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.

Posted by sh...@apache.org.
YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.


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

Branch: refs/heads/HDFS-12943
Commit: 9c3fc3ef2865164aa5f121793ac914cfeb21a181
Parents: 54d0bf8
Author: Weiwei Yang <ww...@apache.org>
Authored: Tue Aug 21 22:42:23 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Tue Aug 21 22:42:28 2018 +0800

----------------------------------------------------------------------
 .../resourcemanager/RMActiveServiceContext.java |  16 ++
 .../yarn/server/resourcemanager/RMContext.java  |   8 +-
 .../server/resourcemanager/RMContextImpl.java   |  14 +-
 .../server/resourcemanager/ResourceManager.java |  12 ++
 .../scheduler/AppSchedulingInfo.java            |  11 +-
 .../scheduler/ClusterNodeTracker.java           |  61 +++++++
 .../scheduler/activities/ActivitiesLogger.java  |  32 ++--
 .../scheduler/activities/ActivitiesManager.java |   8 +-
 .../scheduler/capacity/AbstractCSQueue.java     |  16 +-
 .../scheduler/capacity/CSQueue.java             |   6 +
 .../scheduler/capacity/CapacityScheduler.java   |  77 ++++++++-
 .../CapacitySchedulerConfiguration.java         | 116 +++++++++++++
 .../scheduler/capacity/LeafQueue.java           |  49 +++---
 .../scheduler/capacity/ParentQueue.java         |   4 +-
 .../allocator/RegularContainerAllocator.java    |  35 ++--
 .../common/ApplicationSchedulingConfig.java     |   4 +
 .../scheduler/common/fica/FiCaSchedulerApp.java |  23 +++
 .../LocalityAppPlacementAllocator.java          |  34 +++-
 .../placement/MultiNodeLookupPolicy.java        |  67 ++++++++
 .../placement/MultiNodePolicySpec.java          |  56 +++++++
 .../scheduler/placement/MultiNodeSorter.java    | 167 +++++++++++++++++++
 .../placement/MultiNodeSortingManager.java      | 139 +++++++++++++++
 .../ResourceUsageMultiNodeLookupPolicy.java     |  79 +++++++++
 .../reservation/ReservationSystemTestUtil.java  |   3 +
 .../scheduler/TestAppSchedulingInfo.java        |   3 +-
 .../capacity/CapacitySchedulerTestBase.java     |  13 ++
 .../capacity/TestCapacityScheduler.java         |  15 --
 .../TestCapacitySchedulerMultiNodes.java        | 166 ++++++++++++++++++
 .../TestCapacitySchedulerNodeLabelUpdate.java   |  70 ++++++++
 29 files changed, 1211 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 66065e3..8fb0de6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -43,9 +43,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -113,6 +115,7 @@ public class RMActiveServiceContext {
   private AllocationTagsManager allocationTagsManager;
   private PlacementConstraintManager placementConstraintManager;
   private ResourceProfilesManager resourceProfilesManager;
+  private MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -443,6 +446,19 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
+  public MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager() {
+    return multiNodeSortingManager;
+  }
+
+  @Private
+  @Unstable
+  public void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager) {
+    this.multiNodeSortingManager = multiNodeSortingManager;
+  }
+
+  @Private
+  @Unstable
   public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
     this.schedulerRecoveryStartTime = systemClock.getTime();
     this.schedulerRecoveryWaitTime = waitTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index eb91a31..a30ff76 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -42,10 +42,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -177,4 +178,9 @@ public interface RMContext extends ApplicationMasterServiceContext {
 
   void setPlacementConstraintManager(
       PlacementConstraintManager placementConstraintManager);
+
+  MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager();
+
+  void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 84e0f6f..cb1d56f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -48,10 +48,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -538,6 +539,17 @@ public class RMContextImpl implements RMContext {
         delegatedNodeLabelsUpdater);
   }
 
+  @Override
+  public MultiNodeSortingManager<SchedulerNode> getMultiNodeSortingManager() {
+    return activeServiceContext.getMultiNodeSortingManager();
+  }
+
+  @Override
+  public void setMultiNodeSortingManager(
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager) {
+    activeServiceContext.setMultiNodeSortingManager(multiNodeSortingManager);
+  }
+
   public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
     activeServiceContext.setSchedulerRecoveryStartAndWaitTime(waitTime);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index d459f0e..bdda871 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -96,11 +96,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.MemoryPlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
@@ -546,6 +548,10 @@ public class ResourceManager extends CompositeService
     return new FederationStateStoreService(rmContext);
   }
 
+  protected MultiNodeSortingManager<SchedulerNode> createMultiNodeSortingManager() {
+    return new MultiNodeSortingManager<SchedulerNode>();
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     List<SystemMetricsPublisher> publishers =
         new ArrayList<SystemMetricsPublisher>();
@@ -665,6 +671,12 @@ public class ResourceManager extends CompositeService
       resourceProfilesManager.init(conf);
       rmContext.setResourceProfilesManager(resourceProfilesManager);
 
+      MultiNodeSortingManager<SchedulerNode> multiNodeSortingManager =
+          createMultiNodeSortingManager();
+      multiNodeSortingManager.setRMContext(rmContext);
+      addService(multiNodeSortingManager);
+      rmContext.setMultiNodeSortingManager(multiNodeSortingManager);
+
       RMDelegatedNodeLabelsUpdater delegatedNodeLabelsUpdater =
           createRMDelegatedNodeLabelsUpdater();
       if (delegatedNodeLabelsUpdater != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index d63d2b82..ca7d9ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -93,7 +93,7 @@ public class AppSchedulingInfo {
   private final ReentrantReadWriteLock.WriteLock writeLock;
 
   public final ContainerUpdateContext updateContext;
-  public final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
+  private final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
   private final RMContext rmContext;
 
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
@@ -782,4 +782,13 @@ public class AppSchedulingInfo {
       this.readLock.unlock();
     }
   }
+
+  /**
+   * Get scheduling envs configured for this application.
+   *
+   * @return a map of applicationSchedulingEnvs
+   */
+  public Map<String, String> getApplicationSchedulingEnvs() {
+    return applicationSchedulingEnvs;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
index 66d8810..8c7e447 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ClusterNodeTracker.java
@@ -37,6 +37,7 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -57,6 +58,7 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
   private HashMap<NodeId, N> nodes = new HashMap<>();
   private Map<String, N> nodeNameToNodeMap = new HashMap<>();
   private Map<String, List<N>> nodesPerRack = new HashMap<>();
+  private Map<String, List<N>> nodesPerLabel = new HashMap<>();
 
   private Resource clusterCapacity = Resources.createResource(0, 0);
   private volatile Resource staleClusterCapacity =
@@ -80,6 +82,16 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
       nodes.put(node.getNodeID(), node);
       nodeNameToNodeMap.put(node.getNodeName(), node);
 
+      List<N> nodesPerLabels = nodesPerLabel.get(node.getPartition());
+
+      if (nodesPerLabels == null) {
+        nodesPerLabels = new ArrayList<N>();
+      }
+      nodesPerLabels.add(node);
+
+      // Update new set of nodes for given partition.
+      nodesPerLabel.put(node.getPartition(), nodesPerLabels);
+
       // Update nodes per rack as well
       String rackName = node.getRackName();
       List<N> nodesList = nodesPerRack.get(rackName);
@@ -174,6 +186,16 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
         }
       }
 
+      List<N> nodesPerPartition = nodesPerLabel.get(node.getPartition());
+      nodesPerPartition.remove(node);
+
+      // Update new set of nodes for given partition.
+      if (nodesPerPartition.isEmpty()) {
+        nodesPerLabel.remove(node.getPartition());
+      } else {
+        nodesPerLabel.put(node.getPartition(), nodesPerPartition);
+      }
+
       // Update cluster capacity
       Resources.subtractFrom(clusterCapacity, node.getTotalResource());
       staleClusterCapacity = Resources.clone(clusterCapacity);
@@ -420,4 +442,43 @@ public class ClusterNodeTracker<N extends SchedulerNode> {
     }
     return retNodes;
   }
+
+  /**
+   * update cached nodes per partition on a node label change event.
+   * @param partition nodeLabel
+   * @param nodeIds List of Node IDs
+   */
+  public void updateNodesPerPartition(String partition, Set<NodeId> nodeIds) {
+    writeLock.lock();
+    try {
+      // Clear all entries.
+      nodesPerLabel.remove(partition);
+
+      List<N> nodesPerPartition = new ArrayList<N>();
+      for (NodeId nodeId : nodeIds) {
+        N n = getNode(nodeId);
+        if (n != null) {
+          nodesPerPartition.add(n);
+        }
+      }
+
+      // Update new set of nodes for given partition.
+      nodesPerLabel.put(partition, nodesPerPartition);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public List<N> getNodesPerPartition(String partition) {
+    List<N> nodesPerPartition = null;
+    readLock.lock();
+    try {
+      if (nodesPerLabel.containsKey(partition)) {
+        nodesPerPartition = new ArrayList<N>(nodesPerLabel.get(partition));
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return nodesPerPartition;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
index 0c351b6..8a3ffce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesLogger.java
@@ -21,13 +21,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
 /**
  * Utility for logging scheduler activities
@@ -63,7 +63,7 @@ public class ActivitiesLogger {
         SchedulerApplicationAttempt application, Priority priority,
         String diagnostic) {
       String type = "app";
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -84,18 +84,18 @@ public class ActivitiesLogger {
         ActivitiesManager activitiesManager, SchedulerNode node,
         SchedulerApplicationAttempt application, Priority priority,
         String diagnostic, ActivityState appState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
         String type = "container";
         // Add application-container activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getApplicationId().toString(), null,
             priority.toString(), ActivityState.SKIPPED, diagnostic, type);
         type = "app";
         // Add queue-application activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getQueueName(),
             application.getApplicationId().toString(),
             application.getPriority().toString(), ActivityState.SKIPPED,
@@ -121,20 +121,20 @@ public class ActivitiesLogger {
         ActivitiesManager activitiesManager, SchedulerNode node,
         SchedulerApplicationAttempt application, RMContainer updatedContainer,
         ActivityState activityState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
         String type = "container";
         // Add application-container activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getApplicationId().toString(),
             updatedContainer.getContainer().toString(),
             updatedContainer.getContainer().getPriority().toString(),
             activityState, ActivityDiagnosticConstant.EMPTY, type);
         type = "app";
         // Add queue-application activity into specific node allocation.
-        activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
+        activitiesManager.addSchedulingActivityForNode(node,
             application.getQueueName(),
             application.getApplicationId().toString(),
             application.getPriority().toString(), ActivityState.ACCEPTED,
@@ -157,13 +157,15 @@ public class ActivitiesLogger {
      * update.
      */
     public static void startAppAllocationRecording(
-        ActivitiesManager activitiesManager, NodeId nodeId, long currentTime,
+        ActivitiesManager activitiesManager, FiCaSchedulerNode node,
+        long currentTime,
         SchedulerApplicationAttempt application) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
-      activitiesManager.startAppAllocationRecording(nodeId, currentTime,
-          application);
+      activitiesManager
+          .startAppAllocationRecording(node.getNodeID(), currentTime,
+              application);
     }
 
     /*
@@ -208,7 +210,7 @@ public class ActivitiesLogger {
     public static void recordQueueActivity(ActivitiesManager activitiesManager,
         SchedulerNode node, String parentQueueName, String queueName,
         ActivityState state, String diagnostic) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -240,7 +242,7 @@ public class ActivitiesLogger {
     public static void finishAllocatedNodeAllocation(
         ActivitiesManager activitiesManager, SchedulerNode node,
         ContainerId containerId, AllocationState containerState) {
-      if (activitiesManager == null) {
+      if (node == null || activitiesManager == null) {
         return;
       }
       if (activitiesManager.shouldRecordThisNode(node.getNodeID())) {
@@ -277,7 +279,7 @@ public class ActivitiesLogger {
       SchedulerNode node, String parentName, String childName,
       Priority priority, ActivityState state, String diagnostic, String type) {
 
-    activitiesManager.addSchedulingActivityForNode(node.getNodeID(), parentName,
+    activitiesManager.addSchedulingActivityForNode(node, parentName,
         childName, priority != null ? priority.toString() : null, state,
         diagnostic, type);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
index 8498c40..5d96b17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -197,11 +198,12 @@ public class ActivitiesManager extends AbstractService {
   }
 
   // Add queue, application or container activity into specific node allocation.
-  void addSchedulingActivityForNode(NodeId nodeID, String parentName,
+  void addSchedulingActivityForNode(SchedulerNode node, String parentName,
       String childName, String priority, ActivityState state, String diagnostic,
       String type) {
-    if (shouldRecordThisNode(nodeID)) {
-      NodeAllocation nodeAllocation = getCurrentNodeAllocation(nodeID);
+    if (shouldRecordThisNode(node.getNodeID())) {
+      NodeAllocation nodeAllocation = getCurrentNodeAllocation(
+          node.getNodeID());
       nodeAllocation.addAllocationActivity(parentName, childName, priority,
           state, diagnostic, type);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 9c3e98f..2c9f9a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -92,7 +92,8 @@ public abstract class AbstractCSQueue implements CSQueue {
   Set<String> resourceTypes;
   final RMNodeLabelsManager labelManager;
   String defaultLabelExpression;
-  
+  private String multiNodeSortingPolicyName = null;
+
   Map<AccessType, AccessControlList> acls = 
       new HashMap<AccessType, AccessControlList>();
   volatile boolean reservationsContinueLooking;
@@ -414,6 +415,10 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.priority = configuration.getQueuePriority(
           getQueuePath());
 
+      // Update multi-node sorting algorithm for scheduling as configured.
+      setMultiNodeSortingPolicyName(
+          configuration.getMultiNodesSortingAlgorithmPolicy(getQueuePath()));
+
       this.userWeights = getUserWeightsFromHierarchy(configuration);
     } finally {
       writeLock.unlock();
@@ -1259,4 +1264,13 @@ public abstract class AbstractCSQueue implements CSQueue {
       this.writeLock.unlock();
     }
   }
+
+  @Override
+  public String getMultiNodeSortingPolicyName() {
+    return this.multiNodeSortingPolicyName;
+  }
+
+  public void setMultiNodeSortingPolicyName(String policyName) {
+    this.multiNodeSortingPolicyName = policyName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
index 3963dc0..c0c280e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
@@ -430,4 +430,10 @@ public interface CSQueue extends SchedulerQueue<CSQueue> {
    * @return effective max queue capacity
    */
   Resource getEffectiveMaxCapacityDown(String label, Resource factor);
+
+  /**
+   * Get Multi Node scheduling policy name.
+   * @return policy name
+   */
+  String getMultiNodeSortingPolicyName();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index 0b7fe92..dec1301 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -251,6 +252,7 @@ public class CapacityScheduler extends
   private ResourceCommitterService resourceCommitterService;
   private RMNodeLabelsManager labelManager;
   private AppPriorityACLsManager appPriorityACLManager;
+  private boolean multiNodePlacementEnabled;
 
   private static boolean printedVerboseLoggingForAsyncScheduling = false;
 
@@ -391,12 +393,23 @@ public class CapacityScheduler extends
       // Setup how many containers we can allocate for each round
       offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit();
 
+      // Register CS specific multi-node policies to common MultiNodeManager
+      // which will add to a MultiNodeSorter which gives a pre-sorted list of
+      // nodes to scheduler's allocation.
+      multiNodePlacementEnabled = this.conf.getMultiNodePlacementEnabled();
+      if(rmContext.getMultiNodeSortingManager() != null) {
+        rmContext.getMultiNodeSortingManager().registerMultiNodePolicyNames(
+            multiNodePlacementEnabled,
+            this.conf.getMultiNodePlacementPolicies());
+      }
+
       LOG.info("Initialized CapacityScheduler with " + "calculator="
           + getResourceCalculator().getClass() + ", " + "minimumAllocation=<"
           + getMinimumResourceCapability() + ">, " + "maximumAllocation=<"
           + getMaximumResourceCapability() + ">, " + "asynchronousScheduling="
           + scheduleAsynchronously + ", " + "asyncScheduleInterval="
-          + asyncScheduleInterval + "ms");
+          + asyncScheduleInterval + "ms" + ",multiNodePlacementEnabled="
+          + multiNodePlacementEnabled);
     } finally {
       writeLock.unlock();
     }
@@ -1373,18 +1386,23 @@ public class CapacityScheduler extends
         assignment.getAssignmentInformation().getAllocationDetails();
     List<AssignmentInformation.AssignmentDetails> reservations =
         assignment.getAssignmentInformation().getReservationDetails();
+    // Get nodeId from allocated container if incoming argument is null.
+    NodeId updatedNodeid = (nodeId == null)
+        ? allocations.get(allocations.size() - 1).rmContainer.getNodeId()
+        : nodeId;
+
     if (!allocations.isEmpty()) {
       ContainerId allocatedContainerId =
           allocations.get(allocations.size() - 1).containerId;
       String allocatedQueue = allocations.get(allocations.size() - 1).queue;
-      schedulerHealth.updateAllocation(now, nodeId, allocatedContainerId,
+      schedulerHealth.updateAllocation(now, updatedNodeid, allocatedContainerId,
         allocatedQueue);
     }
     if (!reservations.isEmpty()) {
       ContainerId reservedContainerId =
           reservations.get(reservations.size() - 1).containerId;
       String reservedQueue = reservations.get(reservations.size() - 1).queue;
-      schedulerHealth.updateReservation(now, nodeId, reservedContainerId,
+      schedulerHealth.updateReservation(now, updatedNodeid, reservedContainerId,
         reservedQueue);
     }
     schedulerHealth.updateSchedulerReservationCounts(assignment
@@ -1421,6 +1439,23 @@ public class CapacityScheduler extends
             || assignedContainers < maxAssignPerHeartbeat);
   }
 
+  private CandidateNodeSet<FiCaSchedulerNode> getCandidateNodeSet(
+      FiCaSchedulerNode node) {
+    CandidateNodeSet<FiCaSchedulerNode> candidates = null;
+    candidates = new SimpleCandidateNodeSet<>(node);
+    if (multiNodePlacementEnabled) {
+      Map<NodeId, FiCaSchedulerNode> nodesByPartition = new HashMap<>();
+      List<FiCaSchedulerNode> nodes = nodeTracker
+          .getNodesPerPartition(node.getPartition());
+      if (nodes != null && !nodes.isEmpty()) {
+        nodes.forEach(n -> nodesByPartition.put(n.getNodeID(), n));
+        candidates = new SimpleCandidateNodeSet<FiCaSchedulerNode>(
+            nodesByPartition, node.getPartition());
+      }
+    }
+    return candidates;
+  }
+
   /**
    * We need to make sure when doing allocation, Node should be existed
    * And we will construct a {@link CandidateNodeSet} before proceeding
@@ -1432,8 +1467,8 @@ public class CapacityScheduler extends
       int offswitchCount = 0;
       int assignedContainers = 0;
 
-      CandidateNodeSet<FiCaSchedulerNode> candidates =
-          new SimpleCandidateNodeSet<>(node);
+      CandidateNodeSet<FiCaSchedulerNode> candidates = getCandidateNodeSet(
+          node);
       CSAssignment assignment = allocateContainersToNode(candidates,
           withNodeHeartbeat);
       // Only check if we can allocate more container on the same node when
@@ -1599,10 +1634,13 @@ public class CapacityScheduler extends
 
     if (Resources.greaterThan(calculator, getClusterResource(),
         assignment.getResource(), Resources.none())) {
+      FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
+      NodeId nodeId = null;
+      if (node != null) {
+        nodeId = node.getNodeID();
+      }
       if (withNodeHeartbeat) {
-        updateSchedulerHealth(lastNodeUpdateTime,
-            CandidateNodeSetUtils.getSingleNode(candidates).getNodeID(),
-            assignment);
+        updateSchedulerHealth(lastNodeUpdateTime, nodeId, assignment);
       }
       return assignment;
     }
@@ -1681,7 +1719,7 @@ public class CapacityScheduler extends
     // We have two different logics to handle allocation on single node / multi
     // nodes.
     CSAssignment assignment;
-    if (null != node) {
+    if (!multiNodePlacementEnabled) {
       assignment = allocateContainerOnSingleNode(candidates,
           node, withNodeHeartbeat);
     } else{
@@ -1869,12 +1907,21 @@ public class CapacityScheduler extends
       NodeLabelsUpdateSchedulerEvent labelUpdateEvent) {
     try {
       writeLock.lock();
+      Set<String> updateLabels = new HashSet<String>();
       for (Entry<NodeId, Set<String>> entry : labelUpdateEvent
           .getUpdatedNodeToLabels().entrySet()) {
         NodeId id = entry.getKey();
         Set<String> labels = entry.getValue();
+        FiCaSchedulerNode node = nodeTracker.getNode(id);
+
+        if (node != null) {
+          // Update old partition to list.
+          updateLabels.add(node.getPartition());
+        }
         updateLabelsOnNode(id, labels);
+        updateLabels.addAll(labels);
       }
+      refreshLabelToNodeCache(updateLabels);
       Resource clusterResource = getClusterResource();
       getRootQueue().updateClusterResource(clusterResource,
           new ResourceLimits(clusterResource));
@@ -1883,6 +1930,18 @@ public class CapacityScheduler extends
     }
   }
 
+  private void refreshLabelToNodeCache(Set<String> updateLabels) {
+    Map<String, Set<NodeId>> labelMapping = labelManager
+        .getLabelsToNodes(updateLabels);
+    for (String label : updateLabels) {
+      Set<NodeId> nodes = labelMapping.get(label);
+      if (nodes == null) {
+        continue;
+      }
+      nodeTracker.updateNodesPerPartition(label, nodes);
+    }
+  }
+
   private void addNode(RMNode nodeManager) {
     try {
       writeLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
index e8de096..b937ae7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AppPriorityACLConfigurationParser.AppPriorityACLKeyType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.PriorityUtilizationQueueOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy.QueueOrderingPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeLookupPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodePolicySpec;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
@@ -2129,4 +2131,118 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
       break;
     }
   }
+
+  @Private public static final String MULTI_NODE_SORTING_POLICIES =
+      PREFIX + "multi-node-sorting.policy.names";
+
+  @Private public static final String MULTI_NODE_SORTING_POLICY_NAME =
+      PREFIX + "multi-node-sorting.policy";
+
+  /**
+   * resource usage based node sorting algorithm.
+   */
+  public static final String DEFAULT_NODE_SORTING_POLICY = "default";
+  public static final String DEFAULT_NODE_SORTING_POLICY_CLASSNAME
+      = "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy";
+  public static final long DEFAULT_MULTI_NODE_SORTING_INTERVAL = 1000L;
+
+  @Private
+  public static final String MULTI_NODE_PLACEMENT_ENABLED = PREFIX
+      + "multi-node-placement-enabled";
+
+  @Private
+  public static final boolean DEFAULT_MULTI_NODE_PLACEMENT_ENABLED = false;
+
+  public String getMultiNodesSortingAlgorithmPolicy(
+      String queue) {
+
+    String policyName = get(
+        getQueuePrefix(queue) + "multi-node-sorting.policy");
+
+    if (policyName == null) {
+      policyName = get(MULTI_NODE_SORTING_POLICY_NAME);
+    }
+
+    // If node sorting policy is not configured in queue and in cluster level,
+    // it is been assumed that this queue is not enabled with multi-node lookup.
+    if (policyName == null || policyName.isEmpty()) {
+      return null;
+    }
+
+    String policyClassName = get(MULTI_NODE_SORTING_POLICY_NAME + DOT
+        + policyName.trim() + DOT + "class");
+
+    if (policyClassName == null || policyClassName.isEmpty()) {
+      throw new YarnRuntimeException(
+          policyName.trim() + " Class is not configured or not an instance of "
+              + MultiNodeLookupPolicy.class.getCanonicalName());
+    }
+
+    return normalizePolicyName(policyClassName.trim());
+  }
+
+  public boolean getMultiNodePlacementEnabled() {
+    return getBoolean(MULTI_NODE_PLACEMENT_ENABLED,
+        DEFAULT_MULTI_NODE_PLACEMENT_ENABLED);
+  }
+
+  public Set<MultiNodePolicySpec> getMultiNodePlacementPolicies() {
+    String[] policies = getTrimmedStrings(MULTI_NODE_SORTING_POLICIES);
+
+    // In other cases, split the accessibleLabelStr by ","
+    Set<MultiNodePolicySpec> set = new HashSet<MultiNodePolicySpec>();
+    for (String str : policies) {
+      if (!str.trim().isEmpty()) {
+        String policyClassName = get(
+            MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim() + DOT + "class");
+        if (str.trim().equals(DEFAULT_NODE_SORTING_POLICY)) {
+          policyClassName = get(
+              MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim() + DOT + "class",
+              DEFAULT_NODE_SORTING_POLICY_CLASSNAME);
+        }
+
+        // This check is needed as default class name is loaded only for
+        // DEFAULT_NODE_SORTING_POLICY.
+        if (policyClassName == null) {
+          throw new YarnRuntimeException(
+              str.trim() + " Class is not configured or not an instance of "
+                  + MultiNodeLookupPolicy.class.getCanonicalName());
+        }
+        policyClassName = normalizePolicyName(policyClassName.trim());
+        long policySortingInterval = getLong(
+            MULTI_NODE_SORTING_POLICY_NAME + DOT + str.trim()
+                + DOT + "sorting-interval.ms",
+            DEFAULT_MULTI_NODE_SORTING_INTERVAL);
+        if (policySortingInterval < 0) {
+          throw new YarnRuntimeException(
+              str.trim()
+                  + " multi-node policy is configured with invalid"
+                  + " sorting-interval:" + policySortingInterval);
+        }
+        set.add(
+            new MultiNodePolicySpec(policyClassName, policySortingInterval));
+      }
+    }
+
+    return Collections.unmodifiableSet(set);
+  }
+
+  private String normalizePolicyName(String policyName) {
+
+    // Ensure that custom node sorting algorithm class is valid.
+    try {
+      Class<?> nodeSortingPolicyClazz = getClassByName(policyName);
+      if (MultiNodeLookupPolicy.class
+          .isAssignableFrom(nodeSortingPolicyClazz)) {
+        return policyName;
+      } else {
+        throw new YarnRuntimeException(
+            "Class: " + policyName + " not instance of "
+                + MultiNodeLookupPolicy.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate " + "NodesSortingPolicy: " + policyName, e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 366bad0..ffe862f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -53,10 +53,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
-
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
@@ -1036,23 +1032,24 @@ public class LeafQueue extends AbstractCSQueue {
   private CSAssignment allocateFromReservedContainer(Resource clusterResource,
       CandidateNodeSet<FiCaSchedulerNode> candidates,
       ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) {
-    FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
-    if (null == node) {
-      return null;
-    }
-
-    RMContainer reservedContainer = node.getReservedContainer();
-    if (reservedContainer != null) {
-      FiCaSchedulerApp application = getApplication(
-          reservedContainer.getApplicationAttemptId());
-
-      if (null != application) {
-        ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
-            node.getNodeID(), SystemClock.getInstance().getTime(), application);
-        CSAssignment assignment = application.assignContainers(clusterResource,
-            candidates, currentResourceLimits, schedulingMode,
-            reservedContainer);
-        return assignment;
+    // Considering multi-node scheduling, its better to iterate through
+    // all candidates and stop once we get atleast one good node to allocate
+    // where reservation was made earlier. In normal case, there is only one
+    // node and hence there wont be any impact after this change.
+    for (FiCaSchedulerNode node : candidates.getAllNodes().values()) {
+      RMContainer reservedContainer = node.getReservedContainer();
+      if (reservedContainer != null) {
+        FiCaSchedulerApp application = getApplication(
+            reservedContainer.getApplicationAttemptId());
+
+        if (null != application) {
+          ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
+              node, SystemClock.getInstance().getTime(), application);
+          CSAssignment assignment = application.assignContainers(
+              clusterResource, candidates, currentResourceLimits,
+              schedulingMode, reservedContainer);
+          return assignment;
+        }
       }
     }
 
@@ -1114,13 +1111,14 @@ public class LeafQueue extends AbstractCSQueue {
       FiCaSchedulerApp application = assignmentIterator.next();
 
       ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
-          node.getNodeID(), SystemClock.getInstance().getTime(), application);
+          node, SystemClock.getInstance().getTime(), application);
 
       // Check queue max-capacity limit
       Resource appReserved = application.getCurrentReservation();
       if (needAssignToQueueCheck) {
-        if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
-            currentResourceLimits, appReserved, schedulingMode)) {
+        if (!super.canAssignToThisQueue(clusterResource,
+            candidates.getPartition(), currentResourceLimits, appReserved,
+            schedulingMode)) {
           ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
               activitiesManager, node, application, application.getPriority(),
               ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
@@ -1155,7 +1153,8 @@ public class LeafQueue extends AbstractCSQueue {
         userAssignable = false;
       } else {
         userAssignable = canAssignToUser(clusterResource, application.getUser(),
-            userLimit, application, node.getPartition(), currentResourceLimits);
+            userLimit, application, candidates.getPartition(),
+            currentResourceLimits);
         if (!userAssignable && Resources.fitsIn(cul.reservation, appReserved)) {
           cul.canAssign = false;
           cul.reservation = appReserved;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index 2363b88..80549ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -553,8 +553,8 @@ public class ParentQueue extends AbstractCSQueue {
 
       ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
           getParentName(), getQueueName(), ActivityState.REJECTED,
-          ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
-              .getPartition());
+          ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION
+              + candidates.getPartition());
       if (rootQueue) {
         ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
             node);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index a843002..3e337ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -96,11 +96,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
    * headroom, etc.
    */
   private ContainerAllocation preCheckForNodeCandidateSet(
-      Resource clusterResource, CandidateNodeSet<FiCaSchedulerNode> candidates,
+      Resource clusterResource, FiCaSchedulerNode node,
       SchedulingMode schedulingMode, ResourceLimits resourceLimits,
       SchedulerRequestKey schedulerKey) {
     Priority priority = schedulerKey.getPriority();
-    FiCaSchedulerNode node = CandidateNodeSetUtils.getSingleNode(candidates);
 
     PendingAsk offswitchPendingAsk = application.getPendingAsk(schedulerKey,
         ResourceRequest.ANY);
@@ -164,7 +163,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     }
 
     if (!checkHeadroom(clusterResource, resourceLimits, required,
-        candidates.getPartition())) {
+        node.getPartition())) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("cannot allocate required resource=" + required
             + " because of headroom");
@@ -801,20 +800,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     // Do checks before determining which node to allocate
     // Directly return if this check fails.
     ContainerAllocation result;
-    if (reservedContainer == null) {
-      result = preCheckForNodeCandidateSet(clusterResource, candidates,
-          schedulingMode, resourceLimits, schedulerKey);
-      if (null != result) {
-        return result;
-      }
-    } else {
-      // pre-check when allocating reserved container
-      if (application.getOutstandingAsksCount(schedulerKey) == 0) {
-        // Release
-        return new ContainerAllocation(reservedContainer, null,
-            AllocationState.QUEUE_SKIPPED);
-      }
-    }
 
     AppPlacementAllocator<FiCaSchedulerNode> schedulingPS =
         application.getAppSchedulingInfo().getAppPlacementAllocator(
@@ -833,6 +818,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     while (iter.hasNext()) {
       FiCaSchedulerNode node = iter.next();
 
+      if (reservedContainer == null) {
+        result = preCheckForNodeCandidateSet(clusterResource, node,
+            schedulingMode, resourceLimits, schedulerKey);
+        if (null != result) {
+          continue;
+        }
+      } else {
+        // pre-check when allocating reserved container
+        if (application.getOutstandingAsksCount(schedulerKey) == 0) {
+          // Release
+          result = new ContainerAllocation(reservedContainer, null,
+              AllocationState.QUEUE_SKIPPED);
+          continue;
+        }
+      }
+
       result = tryAllocateOnNode(clusterResource, node, schedulingMode,
           resourceLimits, schedulerKey, reservedContainer);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.java
index 1bd3743..06f74de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/ApplicationSchedulingConfig.java
@@ -32,4 +32,8 @@ public class ApplicationSchedulingConfig {
   @InterfaceAudience.Private
   public static final Class<? extends AppPlacementAllocator>
       DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS = LocalityAppPlacementAllocator.class;
+
+  @InterfaceAudience.Private
+  public static final String ENV_MULTI_NODE_SORTING_POLICY_CLASS =
+      "MULTI_NODE_SORTING_POLICY_CLASS";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 6a5af81..4bfdae9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCap
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
@@ -170,10 +171,32 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       rc = scheduler.getResourceCalculator();
     }
 
+    // Update multi-node sorting algorithm to scheduler envs
+    updateMultiNodeSortingPolicy(rmApp);
+
     containerAllocator = new ContainerAllocator(this, rc, rmContext,
         activitiesManager);
   }
 
+  private void updateMultiNodeSortingPolicy(RMApp rmApp) {
+    if (rmApp == null) {
+      return;
+    }
+
+    String queueName = null;
+    if (scheduler instanceof CapacityScheduler) {
+      queueName = getCSLeafQueue().getMultiNodeSortingPolicyName();
+    }
+
+    if (!appSchedulingInfo.getApplicationSchedulingEnvs().containsKey(
+        ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS)
+        && queueName != null) {
+      appSchedulingInfo.getApplicationSchedulingEnvs().put(
+          ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS,
+          queueName);
+    }
+  }
+
   public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index f1df343..9d30e90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -24,11 +24,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -55,6 +58,8 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       new ConcurrentHashMap<>();
   private volatile String primaryRequestedPartition =
       RMNodeLabelsManager.NO_LABEL;
+  private MultiNodeSortingManager<N> multiNodeSortingManager = null;
+  private String multiNodeSortPolicyName;
 
   private final ReentrantReadWriteLock.ReadLock readLock;
   private final ReentrantReadWriteLock.WriteLock writeLock;
@@ -65,6 +70,26 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     writeLock = lock.writeLock();
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
+  public void initialize(AppSchedulingInfo appSchedulingInfo,
+      SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
+    super.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
+    multiNodeSortPolicyName = appSchedulingInfo
+        .getApplicationSchedulingEnvs().get(
+            ApplicationSchedulingConfig.ENV_MULTI_NODE_SORTING_POLICY_CLASS);
+    multiNodeSortingManager = (MultiNodeSortingManager<N>) rmContext
+        .getMultiNodeSortingManager();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "nodeLookupPolicy used for " + appSchedulingInfo
+              .getApplicationId()
+              + " is " + ((multiNodeSortPolicyName != null) ?
+              multiNodeSortPolicyName :
+              ""));
+    }
+  }
+
   @Override
   @SuppressWarnings("unchecked")
   public Iterator<N> getPreferredNodeIterator(
@@ -74,11 +99,16 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
     // in.
 
     N singleNode = CandidateNodeSetUtils.getSingleNode(candidateNodeSet);
-    if (null != singleNode) {
+    if (singleNode != null) {
       return IteratorUtils.singletonIterator(singleNode);
     }
 
-    return IteratorUtils.emptyIterator();
+    // singleNode will be null if Multi-node placement lookup is enabled, and
+    // hence could consider sorting policies.
+    return multiNodeSortingManager.getMultiNodeSortIterator(
+        candidateNodeSet.getAllNodes().values(),
+        candidateNodeSet.getPartition(),
+        multiNodeSortPolicyName);
   }
 
   private boolean hasRequestLabelChanged(ResourceRequest requestOne,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
new file mode 100644
index 0000000..662e34d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
@@ -0,0 +1,67 @@
+/**
+ * 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.server.resourcemanager.scheduler.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * <p>
+ * This class has the following functionality.
+ *
+ * <p>
+ * Provide an interface for MultiNodeLookupPolicy so that different placement
+ * allocator can choose nodes based on need.
+ * </p>
+ */
+public interface MultiNodeLookupPolicy<N extends SchedulerNode> {
+  /**
+   * Get iterator of preferred node depends on requirement and/or availability.
+   *
+   * @param nodes
+   *          List of Nodes
+   * @param partition
+   *          node label
+   *
+   * @return iterator of preferred node
+   */
+  Iterator<N> getPreferredNodeIterator(Collection<N> nodes, String partition);
+
+  /**
+   * Refresh working nodes set for re-ordering based on the algorithm selected.
+   *
+   * @param nodes
+   *          a collection working nm's.
+   */
+  void addAndRefreshNodesSet(Collection<N> nodes, String partition);
+
+  /**
+   * Get sorted nodes per partition.
+   *
+   * @param partition
+   *          node label
+   *
+   * @return collection of sorted nodes
+   */
+  Set<N> getNodesPerPartition(String partition);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
new file mode 100644
index 0000000..8386d78
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodePolicySpec.java
@@ -0,0 +1,56 @@
+/**
+ * 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.server.resourcemanager.scheduler.placement;
+
+/**
+ * MultiNodePolicySpec contains policyName and timeout.
+ */
+public class MultiNodePolicySpec {
+
+  private String policyName;
+  private long sortingInterval;
+
+  public MultiNodePolicySpec(String policyName, long timeout) {
+    this.setSortingInterval(timeout);
+    this.setPolicyName(policyName);
+  }
+
+  public long getSortingInterval() {
+    return sortingInterval;
+  }
+
+  public void setSortingInterval(long timeout) {
+    this.sortingInterval = timeout;
+  }
+
+  public String getPolicyName() {
+    return policyName;
+  }
+
+  public void setPolicyName(String policyName) {
+    this.policyName = policyName;
+  }
+
+  @Override
+  public String toString() {
+    return "MultiNodePolicySpec {" +
+        "policyName='" + policyName + '\'' +
+        ", sortingInterval=" + sortingInterval +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
new file mode 100644
index 0000000..7e27c34
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSorter.java
@@ -0,0 +1,167 @@
+/**
+ * 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.server.resourcemanager.scheduler.placement;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Common node sorting class which will do sorting based on policy spec.
+ * @param <N> extends SchedulerNode.
+ */
+public class MultiNodeSorter<N extends SchedulerNode> extends AbstractService {
+
+  private MultiNodeLookupPolicy<N> multiNodePolicy;
+  private static final Log LOG = LogFactory.getLog(MultiNodeSorter.class);
+
+  // ScheduledExecutorService which schedules the PreemptionChecker to run
+  // periodically.
+  private ScheduledExecutorService ses;
+  private ScheduledFuture<?> handler;
+  private volatile boolean stopped;
+  private RMContext rmContext;
+  private MultiNodePolicySpec policySpec;
+
+  public MultiNodeSorter(RMContext rmContext,
+      MultiNodePolicySpec policy) {
+    super("MultiNodeLookupPolicy");
+    this.rmContext = rmContext;
+    this.policySpec = policy;
+  }
+
+  @VisibleForTesting
+  public synchronized MultiNodeLookupPolicy<N> getMultiNodeLookupPolicy() {
+    return multiNodePolicy;
+  }
+
+  public void serviceInit(Configuration conf) throws Exception {
+    LOG.info("Initializing MultiNodeSorter=" + policySpec.getPolicyName()
+        + ", with sorting interval=" + policySpec.getSortingInterval());
+    initPolicy(policySpec.getPolicyName());
+    super.serviceInit(conf);
+  }
+
+  @SuppressWarnings("unchecked")
+  void initPolicy(String policyName) throws YarnException {
+    Class<?> policyClass;
+    try {
+      policyClass = Class.forName(policyName);
+    } catch (ClassNotFoundException e) {
+      throw new YarnException(
+          "Invalid policy name:" + policyName + e.getMessage());
+    }
+    this.multiNodePolicy = (MultiNodeLookupPolicy<N>) ReflectionUtils
+        .newInstance(policyClass, null);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    LOG.info("Starting SchedulingMonitor=" + getName());
+    assert !stopped : "starting when already stopped";
+    ses = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
+      public Thread newThread(Runnable r) {
+        Thread t = new Thread(r);
+        t.setName(getName());
+        return t;
+      }
+    });
+
+    // Start sorter thread only if sorting interval is a +ve value.
+    if(policySpec.getSortingInterval() != 0) {
+      handler = ses.scheduleAtFixedRate(new SortingThread(),
+          0, policySpec.getSortingInterval(), TimeUnit.MILLISECONDS);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    stopped = true;
+    if (handler != null) {
+      LOG.info("Stop " + getName());
+      handler.cancel(true);
+      ses.shutdown();
+    }
+    super.serviceStop();
+  }
+
+  @SuppressWarnings("unchecked")
+  @VisibleForTesting
+  public void reSortClusterNodes() {
+    Set<String> nodeLabels = new HashSet<>();
+    nodeLabels
+        .addAll(rmContext.getNodeLabelManager().getClusterNodeLabelNames());
+    nodeLabels.add(RMNodeLabelsManager.NO_LABEL);
+    for (String label : nodeLabels) {
+      Map<NodeId, SchedulerNode> nodesByPartition = new HashMap<>();
+      List<SchedulerNode> nodes = ((AbstractYarnScheduler) rmContext
+          .getScheduler()).getNodeTracker().getNodesPerPartition(label);
+      if (nodes != null && !nodes.isEmpty()) {
+        nodes.forEach(n -> nodesByPartition.put(n.getNodeID(), n));
+        multiNodePolicy.addAndRefreshNodesSet(
+            (Collection<N>) nodesByPartition.values(), label);
+      }
+    }
+  }
+
+  private class SortingThread implements Runnable {
+    @Override
+    public void run() {
+      try {
+        reSortClusterNodes();
+      } catch (Throwable t) {
+        // The preemption monitor does not alter structures nor do structures
+        // persist across invocations. Therefore, log, skip, and retry.
+        LOG.error("Exception raised while executing multinode"
+            + " sorter, skip this run..., exception=", t);
+      }
+    }
+  }
+
+  /**
+   * Verify whether sorter thread is running or not.
+   *
+   * @return true if sorter thread is running, false otherwise.
+   */
+  public boolean isSorterThreadRunning() {
+    return (handler != null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
new file mode 100644
index 0000000..e872317
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeSortingManager.java
@@ -0,0 +1,139 @@
+/**
+ * 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.server.resourcemanager.scheduler.placement;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+/**
+ * Node Sorting Manager which runs all sorter threads and policies.
+ * @param <N> extends SchedulerNode
+ */
+public class MultiNodeSortingManager<N extends SchedulerNode>
+    extends AbstractService {
+
+  private static final Log LOG = LogFactory
+      .getLog(MultiNodeSortingManager.class);
+
+  private RMContext rmContext;
+  private Map<String, MultiNodeSorter<N>> runningMultiNodeSorters;
+  private Set<MultiNodePolicySpec> policySpecs = new HashSet<MultiNodePolicySpec>();
+  private Configuration conf;
+  private boolean multiNodePlacementEnabled;
+
+  public MultiNodeSortingManager() {
+    super("MultiNodeSortingManager");
+    this.runningMultiNodeSorters = new ConcurrentHashMap<>();
+  }
+
+  @Override
+  public void serviceInit(Configuration configuration) throws Exception {
+    LOG.info("Initializing NodeSortingService=" + getName());
+    super.serviceInit(configuration);
+    this.conf = configuration;
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    LOG.info("Starting NodeSortingService=" + getName());
+    createAllPolicies();
+    super.serviceStart();
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    for (MultiNodeSorter<N> sorter : runningMultiNodeSorters.values()) {
+      sorter.stop();
+    }
+    super.serviceStop();
+  }
+
+  private void createAllPolicies() {
+    if (!multiNodePlacementEnabled) {
+      return;
+    }
+    for (MultiNodePolicySpec policy : policySpecs) {
+      MultiNodeSorter<N> mon = new MultiNodeSorter<N>(rmContext, policy);
+      mon.init(conf);
+      mon.start();
+      runningMultiNodeSorters.put(policy.getPolicyName(), mon);
+    }
+  }
+
+  public MultiNodeSorter<N> getMultiNodePolicy(String name) {
+    return runningMultiNodeSorters.get(name);
+  }
+
+  public void setRMContext(RMContext context) {
+    this.rmContext = context;
+  }
+
+  public void registerMultiNodePolicyNames(
+      boolean isMultiNodePlacementEnabled,
+      Set<MultiNodePolicySpec> multiNodePlacementPolicies) {
+    this.policySpecs.addAll(multiNodePlacementPolicies);
+    this.multiNodePlacementEnabled = isMultiNodePlacementEnabled;
+    LOG.info("MultiNode scheduling is '" + multiNodePlacementEnabled +
+        "', and configured policies are " + StringUtils
+        .join(policySpecs.iterator(), ","));
+  }
+
+  public Iterator<N> getMultiNodeSortIterator(Collection<N> nodes,
+      String partition, String policyName) {
+    // nodeLookupPolicy can be null if app is configured with invalid policy.
+    // in such cases, use the the first node.
+    if(policyName == null) {
+      LOG.warn("Multi Node scheduling is enabled, however invalid class is"
+          + " configured. Valid sorting policy has to be configured in"
+          + " yarn.scheduler.capacity.<queue>.multi-node-sorting.policy");
+      return IteratorUtils.singletonIterator(
+          nodes.iterator().next());
+    }
+
+    MultiNodeSorter multiNodeSorter = getMultiNodePolicy(policyName);
+    if (multiNodeSorter == null) {
+      LOG.warn(
+          "MultiNode policy '" + policyName + "' is configured, however " +
+              "yarn.scheduler.capacity.multi-node-placement-enabled is false");
+      return IteratorUtils.singletonIterator(
+          nodes.iterator().next());
+    }
+
+    MultiNodeLookupPolicy<N> policy = multiNodeSorter
+        .getMultiNodeLookupPolicy();
+    // If sorter thread is not running, refresh node set.
+    if (!multiNodeSorter.isSorterThreadRunning()) {
+      policy.addAndRefreshNodesSet(nodes, partition);
+    }
+
+    return policy.getPreferredNodeIterator(nodes, partition);
+  }
+}


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


[33/50] [abbrv] hadoop git commit: HDDS-342. Add example byteman script to print out hadoop rpc traffic. Contributed by Elek, Marton.

Posted by sh...@apache.org.
HDDS-342. Add example byteman script to print out hadoop rpc traffic.
Contributed by Elek, Marton.


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

Branch: refs/heads/HDFS-12943
Commit: af4b705b5f73b177be24292d8dda3a150aa12596
Parents: 4c25f37
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Aug 22 14:48:22 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed Aug 22 14:48:22 2018 -0700

----------------------------------------------------------------------
 dev-support/byteman/README.md                   | 31 ++++++++++++++
 dev-support/byteman/hadooprpc.btm               | 44 ++++++++++++++++++++
 .../src/main/compose/ozone/docker-config        |  2 +
 3 files changed, 77 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/dev-support/byteman/README.md
----------------------------------------------------------------------
diff --git a/dev-support/byteman/README.md b/dev-support/byteman/README.md
new file mode 100644
index 0000000..9a17fc5
--- /dev/null
+++ b/dev-support/byteman/README.md
@@ -0,0 +1,31 @@
+<!--
+  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.
+-->
+
+This folder contains example byteman scripts (http://byteman.jboss.org/) to help 
+Hadoop debuging.
+
+As the startup script of the hadoop-runner docker image supports byteman 
+instrumentation it's enough to set the URL of a script to a specific environment
+variable to activate it with the docker runs:
+
+
+```
+BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm
+```
+
+For more info see HADOOP-15656 and HDDS-342
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/dev-support/byteman/hadooprpc.btm
----------------------------------------------------------------------
diff --git a/dev-support/byteman/hadooprpc.btm b/dev-support/byteman/hadooprpc.btm
new file mode 100644
index 0000000..13894fe
--- /dev/null
+++ b/dev-support/byteman/hadooprpc.btm
@@ -0,0 +1,44 @@
+# 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.
+
+#
+# This script instruments hadoop rpc layer to print out all the request/response messages to the standard output.
+#
+
+RULE Hadoop RPC request
+INTERFACE ^com.google.protobuf.BlockingService
+METHOD callBlockingMethod
+IF true
+DO traceln("--> RPC message request: " + $3.getClass().getSimpleName() + " from " + linked(Thread.currentThread(), "source")); 
+   traceln($3.toString())
+ENDRULE
+
+
+RULE Hadoop RPC response
+INTERFACE ^com.google.protobuf.BlockingService
+METHOD callBlockingMethod
+AT EXIT
+IF true
+DO traceln("--> RPC message response: " + $3.getClass().getSimpleName() + " to " + unlink(Thread.currentThread(), "source")); 
+   traceln($!.toString())
+ENDRULE
+
+
+RULE Hadoop RPC source IP
+CLASS org.apache.hadoop.ipc.Server$RpcCall
+METHOD run
+IF true
+DO link(Thread.currentThread(), "source", $0.connection.toString())
+ENDRULE

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af4b705b/hadoop-dist/src/main/compose/ozone/docker-config
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozone/docker-config b/hadoop-dist/src/main/compose/ozone/docker-config
index 1b75c01..a1828a3 100644
--- a/hadoop-dist/src/main/compose/ozone/docker-config
+++ b/hadoop-dist/src/main/compose/ozone/docker-config
@@ -29,3 +29,5 @@ LOG4J.PROPERTIES_log4j.rootLogger=INFO, stdout
 LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
+#Enable this variable to print out all hadoop rpc traffic to the stdout. See http://byteman.jboss.org/ to define your own instrumentation.
+#BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm


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


[04/50] [abbrv] hadoop git commit: HADOOP-8807. Update README and website to reflect HADOOP-8662. Contributed by Andras Bokor.

Posted by sh...@apache.org.
HADOOP-8807. Update README and website to reflect HADOOP-8662. Contributed by Andras Bokor.


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

Branch: refs/heads/HDFS-12943
Commit: 77b015000a48545209928e31630adaaf6960b4c5
Parents: c67b065
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 17 11:07:23 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 17 11:10:29 2018 +0200

----------------------------------------------------------------------
 README.txt                                                         | 2 +-
 .../src/main/java/org/apache/hadoop/util/ServletUtil.java          | 2 +-
 .../hadoop-hdfs/src/main/native/docs/libhdfs_footer.html           | 2 +-
 hadoop-mapreduce-project/pom.xml                                   | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77b01500/README.txt
----------------------------------------------------------------------
diff --git a/README.txt b/README.txt
index 148cd31..559099b 100644
--- a/README.txt
+++ b/README.txt
@@ -1,6 +1,6 @@
 For the latest information about Hadoop, please visit our website at:
 
-   http://hadoop.apache.org/core/
+   http://hadoop.apache.org/
 
 and our wiki, at:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77b01500/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
index 2fd9b55..9ba9e94 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java
@@ -74,7 +74,7 @@ public class ServletUtil {
   }
 
   public static final String HTML_TAIL = "<hr />\n"
-    + "<a href='http://hadoop.apache.org/core'>Hadoop</a>, "
+    + "<a href='http://hadoop.apache.org'>Hadoop</a>, "
     + Calendar.getInstance().get(Calendar.YEAR) + ".\n"
     + "</body></html>";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77b01500/hadoop-hdfs-project/hadoop-hdfs/src/main/native/docs/libhdfs_footer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/docs/libhdfs_footer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/docs/libhdfs_footer.html
index cb69b63..35930c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/docs/libhdfs_footer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/docs/libhdfs_footer.html
@@ -21,5 +21,5 @@
 <br />
 <br />
 <a href="http://wiki.apache.org/hadoop/LibHDFS">libhdfs</a> - 
-<a href="http://hadoop.apache.org/core/">Hadoop</a> 
+<a href="http://hadoop.apache.org/">Hadoop</a>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77b01500/hadoop-mapreduce-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/pom.xml b/hadoop-mapreduce-project/pom.xml
index e75461a..fe89729 100644
--- a/hadoop-mapreduce-project/pom.xml
+++ b/hadoop-mapreduce-project/pom.xml
@@ -25,7 +25,7 @@
   <version>3.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Hadoop MapReduce</name>
-  <url>http://hadoop.apache.org/mapreduce/</url>
+  <url>http://hadoop.apache.org/</url>
 
   <properties>
     <test.logs>true</test.logs>


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


[36/50] [abbrv] hadoop git commit: HADOOP-14314. The OpenSolaris taxonomy link is dead in InterfaceClassification.md. Contributed by Rui Gao.

Posted by sh...@apache.org.
HADOOP-14314. The OpenSolaris taxonomy link is dead in InterfaceClassification.md. Contributed by Rui Gao.


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

Branch: refs/heads/HDFS-12943
Commit: bed8cb6979e0460141ed77e3b15d4f18db098a8e
Parents: 1ac0144
Author: Takanobu Asanuma <ta...@apache.org>
Authored: Thu Aug 23 21:54:38 2018 +0900
Committer: Takanobu Asanuma <ta...@apache.org>
Committed: Thu Aug 23 21:54:38 2018 +0900

----------------------------------------------------------------------
 .../src/site/markdown/InterfaceClassification.md              | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bed8cb69/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index 451f9be..a21e28b 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -35,10 +35,9 @@ Interface Classification
 ------------------------
 
 Hadoop adopts the following interface classification,
-this classification was derived from the
-[OpenSolaris taxonomy](http://www.opensolaris.org/os/community/arc/policies/interface-taxonomy/#Advice)
-and, to some extent, from taxonomy used inside Yahoo.
-Interfaces have two main attributes: Audience and Stability
+this classification was derived from the OpenSolaris taxonomy and, to some extent,
+from taxonomy used inside Yahoo.
+Interfaces have two main attributes: Audience and Stability.
 
 ### Audience
 


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


[49/50] [abbrv] hadoop git commit: Merge branch 'trunk' into HDFS-12943

Posted by sh...@apache.org.
Merge branch 'trunk' into HDFS-12943


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

Branch: refs/heads/HDFS-12943
Commit: ebd3d6abd4efa8d9ff4c4b5c293206ab2481bfdf
Parents: 8aeb2d8 a4121c7
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:29:34 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:29:34 2018 -0700

----------------------------------------------------------------------
 .../hadoop/io/retry/FailoverProxyProvider.java  |  15 +-
 .../apache/hadoop/hdds/scm/ScmConfigKeys.java   |   6 +-
 .../common/src/main/resources/ozone-default.xml |   7 +-
 .../container/common/impl/ContainerData.java    |   2 +-
 .../statemachine/DatanodeStateMachine.java      |   5 +-
 .../common/statemachine/StateContext.java       |  20 +++
 .../states/datanode/RunningDatanodeState.java   |   1 +
 .../states/endpoint/RegisterEndpointTask.java   |  28 +++-
 .../keyvalue/KeyValueContainerData.java         |  10 +-
 .../container/keyvalue/KeyValueHandler.java     |   7 +-
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |   9 +-
 .../hdds/scm/container/ContainerMapping.java    |  33 ++--
 .../scm/container/ContainerReportHandler.java   |   3 +-
 .../scm/container/ContainerStateManager.java    |   9 +-
 .../hadoop/hdds/scm/container/Mapping.java      |   2 +-
 .../hdds/scm/pipelines/PipelineSelector.java    |   9 +-
 .../scm/server/SCMDatanodeProtocolServer.java   |   2 +-
 .../TestCloseContainerEventHandler.java         |   8 +-
 .../scm/container/TestContainerMapping.java     |  19 ++-
 .../container/closer/TestContainerCloser.java   |   9 +-
 .../ozone/container/common/TestEndPoint.java    |   3 +-
 .../org/apache/hadoop/hdds/scm/cli/SCMCLI.java  |   8 +-
 .../ha/AbstractNNFailoverProxyProvider.java     | 152 ++++++++++++++++++-
 .../ha/ConfiguredFailoverProxyProvider.java     | 141 ++---------------
 .../namenode/ha/IPFailoverProxyProvider.java    |  51 +------
 .../commandhandler/TestBlockDeletion.java       |   3 +-
 .../TestCloseContainerHandler.java              |   4 +-
 .../TestReplicateContainerHandler.java          |   5 +-
 .../yarn/sls/scheduler/SchedulerMetrics.java    |   7 +-
 29 files changed, 322 insertions(+), 256 deletions(-)
----------------------------------------------------------------------



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


[05/50] [abbrv] hadoop git commit: HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun.

Posted by sh...@apache.org.
HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun.


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

Branch: refs/heads/HDFS-12943
Commit: fa121eb66bc42e9cb5586f8c2e268cfdc2ed187a
Parents: 77b0150
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Fri Aug 17 15:22:55 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Fri Aug 17 15:22:55 2018 +0530

----------------------------------------------------------------------
 .../federation/router/RouterRpcServer.java      | 1360 ++----------------
 1 file changed, 158 insertions(+), 1202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa121eb6/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 29f32a6..fe54993 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -33,16 +33,12 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -54,7 +50,6 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -64,7 +59,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
@@ -93,7 +87,6 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
-import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -101,8 +94,8 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
@@ -167,11 +160,6 @@ public class RouterRpcServer extends AbstractService
   /** Configuration for the RPC server. */
   private Configuration conf;
 
-  /** Identifier for the super user. */
-  private final String superUser;
-  /** Identifier for the super group. */
-  private final String superGroup;
-
   /** Router using this RPC server. */
   private final Router router;
 
@@ -199,11 +187,10 @@ public class RouterRpcServer extends AbstractService
   // Modules implementing groups of RPC calls
   /** Router Quota calls. */
   private final Quota quotaCall;
-  /** Erasure coding calls. */
-  private final ErasureCoding erasureCoding;
   /** NamenodeProtocol calls. */
   private final RouterNamenodeProtocol nnProto;
-
+  /** ClientProtocol calls. */
+  private final RouterClientProtocol clientProto;
 
   /**
    * Construct a router RPC server.
@@ -223,12 +210,6 @@ public class RouterRpcServer extends AbstractService
     this.namenodeResolver = nnResolver;
     this.subclusterResolver = fileResolver;
 
-    // User and group for reporting
-    this.superUser = System.getProperty("user.name");
-    this.superGroup = this.conf.get(
-        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
-        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
-
     // RPC server settings
     int handlerCount = this.conf.getInt(DFS_ROUTER_HANDLER_COUNT_KEY,
         DFS_ROUTER_HANDLER_COUNT_DEFAULT);
@@ -315,8 +296,8 @@ public class RouterRpcServer extends AbstractService
 
     // Initialize modules
     this.quotaCall = new Quota(this.router, this);
-    this.erasureCoding = new ErasureCoding(this);
     this.nnProto = new RouterNamenodeProtocol(this);
+    this.clientProto = new RouterClientProtocol(conf, this);
   }
 
   @Override
@@ -371,6 +352,13 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
+   * Get the active namenode resolver
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return namenodeResolver;
+  }
+
+  /**
    * Get the RPC monitor and metrics.
    *
    * @return RPC monitor and metrics.
@@ -411,7 +399,7 @@ public class RouterRpcServer extends AbstractService
    *                           client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  protected void checkOperation(OperationCategory op, boolean supported)
+  void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -433,7 +421,7 @@ public class RouterRpcServer extends AbstractService
    * @throws SafeModeException If the Router is in safe mode and cannot serve
    *                           client requests.
    */
-  protected void checkOperation(OperationCategory op)
+  void checkOperation(OperationCategory op)
       throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
@@ -464,58 +452,44 @@ public class RouterRpcServer extends AbstractService
     }
   }
 
+  /**
+   * Get the name of the method that is calling this function.
+   *
+   * @return Name of the method calling this function.
+   */
+  static String getMethodName() {
+    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
+    String methodName = stack[3].getMethodName();
+    return methodName;
+  }
+
   @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
-  }
-
-  /**
-   * The the delegation token from each name service.
-   * @param renewer
-   * @return Name service -> Token.
-   * @throws IOException
-   */
-  public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
-      getDelegationTokens(Text renewer) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
+    return clientProto.getDelegationToken(renewer);
   }
 
   @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return 0;
+    return clientProto.renewDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.cancelDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public LocatedBlocks getBlockLocations(String src, final long offset,
       final long length) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
-        new Class<?>[] {String.class, long.class, long.class},
-        new RemoteParam(), offset, length);
-    return (LocatedBlocks) rpcClient.invokeSequential(locations, remoteMethod,
-        LocatedBlocks.class, null);
+    return clientProto.getBlockLocations(src, offset, length);
   }
 
   @Override // ClientProtocol
   public FsServerDefaults getServerDefaults() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("getServerDefaults");
-    String ns = subclusterResolver.getDefaultNamespace();
-    return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
+    return clientProto.getServerDefaults();
   }
 
   @Override // ClientProtocol
@@ -524,44 +498,8 @@ public class RouterRpcServer extends AbstractService
       boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    if (createParent && isPathAll(src)) {
-      int index = src.lastIndexOf(Path.SEPARATOR);
-      String parent = src.substring(0, index);
-      LOG.debug("Creating {} requires creating parent {}", src, parent);
-      FsPermission parentPermissions = getParentPermission(masked);
-      boolean success = mkdirs(parent, parentPermissions, createParent);
-      if (!success) {
-        // This shouldn't happen as mkdirs returns true or exception
-        LOG.error("Couldn't create parents for {}", src);
-      }
-    }
-
-    RemoteLocation createLocation = getCreateLocation(src);
-    RemoteMethod method = new RemoteMethod("create",
-        new Class<?>[] {String.class, FsPermission.class, String.class,
-                        EnumSetWritable.class, boolean.class, short.class,
-                        long.class, CryptoProtocolVersion[].class,
-                        String.class},
-        createLocation.getDest(), masked, clientName, flag, createParent,
+    return clientProto.create(src, masked, clientName, flag, createParent,
         replication, blockSize, supportedVersions, ecPolicyName);
-    return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
-  }
-
-  /**
-   * Get the permissions for the parent of a child with given permissions.
-   * Add implicit u+wx permission for parent. This is based on
-   * @{FSDirMkdirOp#addImplicitUwx}.
-   * @param mask The permission mask of the child.
-   * @return The permission mask of the parent.
-   */
-  private static FsPermission getParentPermission(final FsPermission mask) {
-    FsPermission ret = new FsPermission(
-        mask.getUserAction().or(FsAction.WRITE_EXECUTE),
-        mask.getGroupAction(),
-        mask.getOtherAction());
-    return ret;
   }
 
   /**
@@ -572,7 +510,7 @@ public class RouterRpcServer extends AbstractService
    * @return The remote location for this file.
    * @throws IOException If the file has no creation location.
    */
-  protected RemoteLocation getCreateLocation(final String src)
+  RemoteLocation getCreateLocation(final String src)
       throws IOException {
 
     final List<RemoteLocation> locations = getLocationsForPath(src, true);
@@ -613,100 +551,45 @@ public class RouterRpcServer extends AbstractService
     return createLocation;
   }
 
-  // Medium
   @Override // ClientProtocol
   public LastBlockWithStatus append(String src, final String clientName,
       final EnumSetWritable<CreateFlag> flag) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("append",
-        new Class<?>[] {String.class, String.class, EnumSetWritable.class},
-        new RemoteParam(), clientName, flag);
-    return rpcClient.invokeSequential(
-        locations, method, LastBlockWithStatus.class, null);
+    return clientProto.append(src, clientName, flag);
   }
 
-  // Low
   @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("recoverLease",
-        new Class<?>[] {String.class, String.class}, new RemoteParam(),
-        clientName);
-    Object result = rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE);
-    return (boolean) result;
+    return clientProto.recoverLease(src, clientName);
   }
 
   @Override // ClientProtocol
   public boolean setReplication(String src, short replication)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setReplication",
-        new Class<?>[] {String.class, short.class}, new RemoteParam(),
-        replication);
-    Object result = rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE);
-    return (boolean) result;
+    return clientProto.setReplication(src, replication);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setStoragePolicy",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), policyName);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.setStoragePolicy(src, policyName);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("getStoragePolicies");
-    String ns = subclusterResolver.getDefaultNamespace();
-    return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
+    return clientProto.getStoragePolicies();
   }
 
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setPermission",
-        new Class<?>[] {String.class, FsPermission.class},
-        new RemoteParam(), permissions);
-    if (isPathAll(src)) {
-      rpcClient.invokeConcurrent(locations, method);
-    } else {
-      rpcClient.invokeSequential(locations, method);
-    }
+    clientProto.setPermission(src, permissions);
   }
 
   @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setOwner",
-        new Class<?>[] {String.class, String.class, String.class},
-        new RemoteParam(), username, groupname);
-    if (isPathAll(src)) {
-      rpcClient.invokeConcurrent(locations, method);
-    } else {
-      rpcClient.invokeSequential(locations, method);
-    }
+    clientProto.setOwner(src, username, groupname);
   }
 
   /**
@@ -718,18 +601,8 @@ public class RouterRpcServer extends AbstractService
       ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
       String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("addBlock",
-        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
-                        DatanodeInfo[].class, long.class, String[].class,
-                        EnumSet.class},
-        new RemoteParam(), clientName, previous, excludedNodes, fileId,
-        favoredNodes, addBlockFlags);
-    // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
-    return (LocatedBlock) rpcClient.invokeSequential(
-        locations, method, LocatedBlock.class, null);
+    return clientProto.addBlock(src, clientName, previous, excludedNodes,
+        fileId, favoredNodes, addBlockFlags);
   }
 
   /**
@@ -742,55 +615,26 @@ public class RouterRpcServer extends AbstractService
       final String[] existingStorageIDs, final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName)
           throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
-        new Class<?>[] {String.class, long.class, ExtendedBlock.class,
-                        DatanodeInfo[].class, String[].class,
-                        DatanodeInfo[].class, int.class, String.class},
-        new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
-        numAdditionalNodes, clientName);
-    return (LocatedBlock) rpcClient.invokeSequential(
-        locations, method, LocatedBlock.class, null);
+    return clientProto.getAdditionalDatanode(src, fileId, blk, existings,
+        existingStorageIDs, excludes, numAdditionalNodes, clientName);
   }
 
   @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
       String holder) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("abandonBlock",
-        new Class<?>[] {ExtendedBlock.class, long.class, String.class,
-                        String.class},
-        b, fileId, new RemoteParam(), holder);
-    rpcClient.invokeSingle(b, method);
+    clientProto.abandonBlock(b, fileId, src, holder);
   }
 
   @Override // ClientProtocol
   public boolean complete(String src, String clientName, ExtendedBlock last,
       long fileId) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("complete",
-        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
-                        long.class},
-        new RemoteParam(), clientName, last, fileId);
-    // Complete can return true/false, so don't expect a result
-    return ((Boolean) rpcClient.invokeSequential(
-        locations, method, Boolean.class, null)).booleanValue();
+    return clientProto.complete(src, clientName, last, fileId);
   }
 
   @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(
       ExtendedBlock block, String clientName) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
-        new Class<?>[] {ExtendedBlock.class, String.class},
-        block, clientName);
-    return (LocatedBlock) rpcClient.invokeSingle(block, method);
+    return clientProto.updateBlockForPipeline(block, clientName);
   }
 
   /**
@@ -801,462 +645,91 @@ public class RouterRpcServer extends AbstractService
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
           throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("updatePipeline",
-        new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
-                        DatanodeID[].class, String[].class},
-        clientName, oldBlock, newBlock, newNodes, newStorageIDs);
-    rpcClient.invokeSingle(oldBlock, method);
+    clientProto.updatePipeline(clientName, oldBlock, newBlock, newNodes,
+        newStorageIDs);
   }
 
   @Override // ClientProtocol
   public long getPreferredBlockSize(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return ((Long) rpcClient.invokeSequential(
-        locations, method, Long.class, null)).longValue();
-  }
-
-  /**
-   * Determines combinations of eligible src/dst locations for a rename. A
-   * rename cannot change the namespace. Renames are only allowed if there is an
-   * eligible dst location in the same namespace as the source.
-   *
-   * @param srcLocations List of all potential source destinations where the
-   *          path may be located. On return this list is trimmed to include
-   *          only the paths that have corresponding destinations in the same
-   *          namespace.
-   * @param dst The destination path
-   * @return A map of all eligible source namespaces and their corresponding
-   *         replacement value.
-   * @throws IOException If the dst paths could not be determined.
-   */
-  private RemoteParam getRenameDestinations(
-      final List<RemoteLocation> srcLocations, final String dst)
-          throws IOException {
-
-    final List<RemoteLocation> dstLocations = getLocationsForPath(dst, true);
-    final Map<RemoteLocation, String> dstMap = new HashMap<>();
-
-    Iterator<RemoteLocation> iterator = srcLocations.iterator();
-    while (iterator.hasNext()) {
-      RemoteLocation srcLocation = iterator.next();
-      RemoteLocation eligibleDst =
-          getFirstMatchingLocation(srcLocation, dstLocations);
-      if (eligibleDst != null) {
-        // Use this dst for this source location
-        dstMap.put(srcLocation, eligibleDst.getDest());
-      } else {
-        // This src destination is not valid, remove from the source list
-        iterator.remove();
-      }
-    }
-    return new RemoteParam(dstMap);
-  }
-
-  /**
-   * Get first matching location.
-   *
-   * @param location Location we are looking for.
-   * @param locations List of locations.
-   * @return The first matchin location in the list.
-   */
-  private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
-      List<RemoteLocation> locations) {
-    for (RemoteLocation loc : locations) {
-      if (loc.getNameserviceId().equals(location.getNameserviceId())) {
-        // Return first matching location
-        return loc;
-      }
-    }
-    return null;
+    return clientProto.getPreferredBlockSize(src);
   }
 
   @Deprecated
   @Override // ClientProtocol
   public boolean rename(final String src, final String dst)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> srcLocations =
-        getLocationsForPath(src, true, false);
-    // srcLocations may be trimmed by getRenameDestinations()
-    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
-    RemoteParam dstParam = getRenameDestinations(locs, dst);
-    if (locs.isEmpty()) {
-      throw new IOException(
-          "Rename of " + src + " to " + dst + " is not allowed," +
-          " no eligible destination in the same namespace was found.");
-    }
-    RemoteMethod method = new RemoteMethod("rename",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), dstParam);
-    return ((Boolean) rpcClient.invokeSequential(
-        locs, method, Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.rename(src, dst);
   }
 
   @Override // ClientProtocol
   public void rename2(final String src, final String dst,
       final Options.Rename... options) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> srcLocations =
-        getLocationsForPath(src, true, false);
-    // srcLocations may be trimmed by getRenameDestinations()
-    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
-    RemoteParam dstParam = getRenameDestinations(locs, dst);
-    if (locs.isEmpty()) {
-      throw new IOException(
-          "Rename of " + src + " to " + dst + " is not allowed," +
-          " no eligible destination in the same namespace was found.");
-    }
-    RemoteMethod method = new RemoteMethod("rename2",
-        new Class<?>[] {String.class, String.class, options.getClass()},
-        new RemoteParam(), dstParam, options);
-    rpcClient.invokeSequential(locs, method, null, null);
+    clientProto.rename2(src, dst, options);
   }
 
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // See if the src and target files are all in the same namespace
-    LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
-    if (targetBlocks == null) {
-      throw new IOException("Cannot locate blocks for target file - " + trg);
-    }
-    LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
-    String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
-    for (String source : src) {
-      LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
-      if (sourceBlocks == null) {
-        throw new IOException(
-            "Cannot located blocks for source file " + source);
-      }
-      String sourceBlockPoolId =
-          sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
-      if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
-        throw new IOException("Cannot concatenate source file " + source
-            + " because it is located in a different namespace"
-            + " with block pool id " + sourceBlockPoolId
-            + " from the target file with block pool id "
-            + targetBlockPoolId);
-      }
-    }
-
-    // Find locations in the matching namespace.
-    final RemoteLocation targetDestination =
-        getLocationForPath(trg, true, targetBlockPoolId);
-    String[] sourceDestinations = new String[src.length];
-    for (int i = 0; i < src.length; i++) {
-      String sourceFile = src[i];
-      RemoteLocation location =
-          getLocationForPath(sourceFile, true, targetBlockPoolId);
-      sourceDestinations[i] = location.getDest();
-    }
-    // Invoke
-    RemoteMethod method = new RemoteMethod("concat",
-        new Class<?>[] {String.class, String[].class},
-        targetDestination.getDest(), sourceDestinations);
-    rpcClient.invokeSingle(targetDestination, method);
+    clientProto.concat(trg, src);
   }
 
   @Override // ClientProtocol
   public boolean truncate(String src, long newLength, String clientName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("truncate",
-        new Class<?>[] {String.class, long.class, String.class},
-        new RemoteParam(), newLength, clientName);
-    return ((Boolean) rpcClient.invokeSequential(locations, method,
-        Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.truncate(src, newLength, clientName);
   }
 
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations =
-        getLocationsForPath(src, true, false);
-    RemoteMethod method = new RemoteMethod("delete",
-        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
-        recursive);
-    if (isPathAll(src)) {
-      return rpcClient.invokeAll(locations, method);
-    } else {
-      return rpcClient.invokeSequential(locations, method,
-          Boolean.class, Boolean.TRUE).booleanValue();
-    }
+    return clientProto.delete(src, recursive);
   }
 
   @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("mkdirs",
-        new Class<?>[] {String.class, FsPermission.class, boolean.class},
-        new RemoteParam(), masked, createParent);
-
-    // Create in all locations
-    if (isPathAll(src)) {
-      return rpcClient.invokeAll(locations, method);
-    }
-
-    if (locations.size() > 1) {
-      // Check if this directory already exists
-      try {
-        HdfsFileStatus fileStatus = getFileInfo(src);
-        if (fileStatus != null) {
-          // When existing, the NN doesn't return an exception; return true
-          return true;
-        }
-      } catch (IOException ioe) {
-        // Can't query if this file exists or not.
-        LOG.error("Error requesting file info for path {} while proxing mkdirs",
-            src, ioe);
-      }
-    }
-
-    RemoteLocation firstLocation = locations.get(0);
-    return ((Boolean) rpcClient.invokeSingle(firstLocation, method))
-        .booleanValue();
+    return clientProto.mkdirs(src, masked, createParent);
   }
 
   @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("renewLease",
-        new Class<?>[] {String.class}, clientName);
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, false, false);
+    clientProto.renewLease(clientName);
   }
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // Locate the dir and fetch the listing
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("getListing",
-        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
-        new RemoteParam(), startAfter, needLocation);
-    Map<RemoteLocation, DirectoryListing> listings =
-        rpcClient.invokeConcurrent(
-            locations, method, false, false, DirectoryListing.class);
-
-    Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
-    int totalRemainingEntries = 0;
-    int remainingEntries = 0;
-    boolean namenodeListingExists = false;
-    if (listings != null) {
-      // Check the subcluster listing with the smallest name
-      String lastName = null;
-      for (Entry<RemoteLocation, DirectoryListing> entry :
-          listings.entrySet()) {
-        RemoteLocation location = entry.getKey();
-        DirectoryListing listing = entry.getValue();
-        if (listing == null) {
-          LOG.debug("Cannot get listing from {}", location);
-        } else {
-          totalRemainingEntries += listing.getRemainingEntries();
-          HdfsFileStatus[] partialListing = listing.getPartialListing();
-          int length = partialListing.length;
-          if (length > 0) {
-            HdfsFileStatus lastLocalEntry = partialListing[length-1];
-            String lastLocalName = lastLocalEntry.getLocalName();
-            if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
-              lastName = lastLocalName;
-            }
-          }
-        }
-      }
-
-      // Add existing entries
-      for (Object value : listings.values()) {
-        DirectoryListing listing = (DirectoryListing) value;
-        if (listing != null) {
-          namenodeListingExists = true;
-          for (HdfsFileStatus file : listing.getPartialListing()) {
-            String filename = file.getLocalName();
-            if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
-              // Discarding entries further than the lastName
-              remainingEntries++;
-            } else {
-              nnListing.put(filename, file);
-            }
-          }
-          remainingEntries += listing.getRemainingEntries();
-        }
-      }
-    }
-
-    // Add mount points at this level in the tree
-    final List<String> children = subclusterResolver.getMountPoints(src);
-    if (children != null) {
-      // Get the dates for each mount point
-      Map<String, Long> dates = getMountPointDates(src);
-
-      // Create virtual folder with the mount name
-      for (String child : children) {
-        long date = 0;
-        if (dates != null && dates.containsKey(child)) {
-          date = dates.get(child);
-        }
-        // TODO add number of children
-        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
-
-        // This may overwrite existing listing entries with the mount point
-        // TODO don't add if already there?
-        nnListing.put(child, dirStatus);
-      }
-    }
-
-    if (!namenodeListingExists && nnListing.size() == 0) {
-      // NN returns a null object if the directory cannot be found and has no
-      // listing. If we didn't retrieve any NN listing data, and there are no
-      // mount points here, return null.
-      return null;
-    }
-
-    // Generate combined listing
-    HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
-    combinedData = nnListing.values().toArray(combinedData);
-    return new DirectoryListing(combinedData, remainingEntries);
+    return clientProto.getListing(src, startAfter, needLocation);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getFileInfo",
-        new Class<?>[] {String.class}, new RemoteParam());
-
-    HdfsFileStatus ret = null;
-    // If it's a directory, we check in all locations
-    if (isPathAll(src)) {
-      ret = getFileInfoAll(locations, method);
-    } else {
-      // Check for file information sequentially
-      ret = (HdfsFileStatus) rpcClient.invokeSequential(
-          locations, method, HdfsFileStatus.class, null);
-    }
-
-    // If there is no real path, check mount points
-    if (ret == null) {
-      List<String> children = subclusterResolver.getMountPoints(src);
-      if (children != null && !children.isEmpty()) {
-        Map<String, Long> dates = getMountPointDates(src);
-        long date = 0;
-        if (dates != null && dates.containsKey(src)) {
-          date = dates.get(src);
-        }
-        ret = getMountPointStatus(src, children.size(), date);
-      }
-    }
-
-    return ret;
-  }
-
-  /**
-   * Get the file info from all the locations.
-   *
-   * @param locations Locations to check.
-   * @param method The file information method to run.
-   * @return The first file info if it's a file, the directory if it's
-   *         everywhere.
-   * @throws IOException If all the locations throw an exception.
-   */
-  private HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
-      final RemoteMethod method) throws IOException {
-
-    // Get the file info from everybody
-    Map<RemoteLocation, HdfsFileStatus> results =
-        rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
-
-    // We return the first file
-    HdfsFileStatus dirStatus = null;
-    for (RemoteLocation loc : locations) {
-      HdfsFileStatus fileStatus = results.get(loc);
-      if (fileStatus != null) {
-        if (!fileStatus.isDirectory()) {
-          return fileStatus;
-        } else if (dirStatus == null) {
-          dirStatus = fileStatus;
-        }
-      }
-    }
-    return dirStatus;
+    return clientProto.getFileInfo(src);
   }
 
   @Override // ClientProtocol
   public boolean isFileClosed(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("isFileClosed",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return ((Boolean) rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.isFileClosed(src);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getFileLinkInfo",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (HdfsFileStatus) rpcClient.invokeSequential(
-        locations, method, HdfsFileStatus.class, null);
+    return clientProto.getFileLinkInfo(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public HdfsLocatedFileStatus getLocatedFileInfo(String src,
       boolean needBlockToken) throws IOException {
-    checkOperation(OperationCategory.READ);
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
-        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
-        Boolean.valueOf(needBlockToken));
-    return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
-        locations, method, HdfsFileStatus.class, null);
+    return clientProto.getLocatedFileInfo(src, needBlockToken);
   }
 
   @Override // ClientProtocol
   public long[] getStats() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("getStats");
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, long[]> results =
-        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
-    long[] combinedData = new long[STATS_ARRAY_LENGTH];
-    for (long[] data : results.values()) {
-      for (int i = 0; i < combinedData.length && i < data.length; i++) {
-        if (data[i] >= 0) {
-          combinedData[i] += data[i];
-        }
-      }
-    }
-    return combinedData;
+    return clientProto.getStats();
   }
 
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-    return getDatanodeReport(type, true, 0);
+    return clientProto.getDatanodeReport(type);
   }
 
   /**
@@ -1305,29 +778,7 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public DatanodeStorageReport[] getDatanodeStorageReport(
       DatanodeReportType type) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    Map<String, DatanodeStorageReport[]> dnSubcluster =
-        getDatanodeStorageReportMap(type);
-
-    // Avoid repeating machines in multiple subclusters
-    Map<String, DatanodeStorageReport> datanodesMap = new LinkedHashMap<>();
-    for (DatanodeStorageReport[] dns : dnSubcluster.values()) {
-      for (DatanodeStorageReport dn : dns) {
-        DatanodeInfo dnInfo = dn.getDatanodeInfo();
-        String nodeId = dnInfo.getXferAddr();
-        if (!datanodesMap.containsKey(nodeId)) {
-          datanodesMap.put(nodeId, dn);
-        }
-        // TODO merge somehow, right now it just takes the first one
-      }
-    }
-
-    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
-    DatanodeStorageReport[] combinedData =
-        new DatanodeStorageReport[datanodes.size()];
-    combinedData = datanodes.toArray(combinedData);
-    return combinedData;
+    return clientProto.getDatanodeStorageReport(type);
   }
 
   /**
@@ -1360,740 +811,388 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action, boolean isChecked)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // Set safe mode in all the name spaces
-    RemoteMethod method = new RemoteMethod("setSafeMode",
-        new Class<?>[] {SafeModeAction.class, boolean.class},
-        action, isChecked);
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> results =
-        rpcClient.invokeConcurrent(
-            nss, method, true, !isChecked, Boolean.class);
-
-    // We only report true if all the name space are in safe mode
-    int numSafemode = 0;
-    for (boolean safemode : results.values()) {
-      if (safemode) {
-        numSafemode++;
-      }
-    }
-    return numSafemode == results.size();
+    return clientProto.setSafeMode(action, isChecked);
   }
 
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("restoreFailedStorage",
-        new Class<?>[] {String.class}, arg);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class);
-
-    boolean success = true;
-    for (boolean s : ret.values()) {
-      if (!s) {
-        success = false;
-        break;
-      }
-    }
-    return success;
+    return clientProto.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("saveNamespace",
-        new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
-
-    boolean success = true;
-    for (boolean s : ret.values()) {
-      if (!s) {
-        success = false;
-        break;
-      }
-    }
-    return success;
+    return clientProto.saveNamespace(timeWindow, txGap);
   }
 
   @Override // ClientProtocol
   public long rollEdits() throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Long> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
-
-    // Return the maximum txid
-    long txid = 0;
-    for (long t : ret.values()) {
-      if (t > txid) {
-        txid = t;
-      }
-    }
-    return txid;
+    return clientProto.rollEdits();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, true);
+    clientProto.refreshNodes();
   }
 
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("finalizeUpgrade",
-        new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public boolean upgradeStatus() throws IOException {
-    String methodName = getMethodName();
-    throw new UnsupportedOperationException(
-        "Operation \"" + methodName + "\" is not supported");
+    return clientProto.upgradeStatus();
   }
 
   @Override // ClientProtocol
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("rollingUpgrade",
-        new Class<?>[] {RollingUpgradeAction.class}, action);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
-        rpcClient.invokeConcurrent(
-            nss, method, true, false, RollingUpgradeInfo.class);
-
-    // Return the first rolling upgrade info
-    RollingUpgradeInfo info = null;
-    for (RollingUpgradeInfo infoNs : ret.values()) {
-      if (info == null && infoNs != null) {
-        info = infoNs;
-      }
-    }
-    return info;
+    return clientProto.rollingUpgrade(action);
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("metaSave",
-        new Class<?>[] {String.class}, filename);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.metaSave(filename);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(path, false);
-    RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), cookie);
-    return (CorruptFileBlocks) rpcClient.invokeSequential(
-        locations, method, CorruptFileBlocks.class, null);
+    return clientProto.listCorruptFileBlocks(path, cookie);
   }
 
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
-        new Class<?>[] {Long.class}, bandwidth);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.setBalancerBandwidth(bandwidth);
   }
 
   @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // Get the summaries from regular files
-    Collection<ContentSummary> summaries = new LinkedList<>();
-    FileNotFoundException notFoundException = null;
-    try {
-      final List<RemoteLocation> locations = getLocationsForPath(path, false);
-      RemoteMethod method = new RemoteMethod("getContentSummary",
-          new Class<?>[] {String.class}, new RemoteParam());
-      Map<RemoteLocation, ContentSummary> results =
-          rpcClient.invokeConcurrent(
-              locations, method, false, false, ContentSummary.class);
-      summaries.addAll(results.values());
-    } catch (FileNotFoundException e) {
-      notFoundException = e;
-    }
-
-    // Add mount points at this level in the tree
-    final List<String> children = subclusterResolver.getMountPoints(path);
-    if (children != null) {
-      for (String child : children) {
-        Path childPath = new Path(path, child);
-        try {
-          ContentSummary mountSummary = getContentSummary(childPath.toString());
-          if (mountSummary != null) {
-            summaries.add(mountSummary);
-          }
-        } catch (Exception e) {
-          LOG.error("Cannot get content summary for mount {}: {}",
-              childPath, e.getMessage());
-        }
-      }
-    }
-
-    // Throw original exception if no original nor mount points
-    if (summaries.isEmpty() && notFoundException != null) {
-      throw notFoundException;
-    }
-
-    return aggregateContentSummary(summaries);
-  }
-
-  /**
-   * Aggregate content summaries for each subcluster.
-   *
-   * @param summaries Collection of individual summaries.
-   * @return Aggregated content summary.
-   */
-  private ContentSummary aggregateContentSummary(
-      Collection<ContentSummary> summaries) {
-    if (summaries.size() == 1) {
-      return summaries.iterator().next();
-    }
-
-    long length = 0;
-    long fileCount = 0;
-    long directoryCount = 0;
-    long quota = 0;
-    long spaceConsumed = 0;
-    long spaceQuota = 0;
-
-    for (ContentSummary summary : summaries) {
-      length += summary.getLength();
-      fileCount += summary.getFileCount();
-      directoryCount += summary.getDirectoryCount();
-      quota += summary.getQuota();
-      spaceConsumed += summary.getSpaceConsumed();
-      spaceQuota += summary.getSpaceQuota();
-    }
-
-    ContentSummary ret = new ContentSummary.Builder()
-        .length(length)
-        .fileCount(fileCount)
-        .directoryCount(directoryCount)
-        .quota(quota)
-        .spaceConsumed(spaceConsumed)
-        .spaceQuota(spaceQuota)
-        .build();
-    return ret;
+    return clientProto.getContentSummary(path);
   }
 
   @Override // ClientProtocol
   public void fsync(String src, long fileId, String clientName,
       long lastBlockLength) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("fsync",
-        new Class<?>[] {String.class, long.class, String.class, long.class },
-        new RemoteParam(), fileId, clientName, lastBlockLength);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.fsync(src, fileId, clientName, lastBlockLength);
   }
 
   @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setTimes",
-        new Class<?>[] {String.class, long.class, long.class},
-        new RemoteParam(), mtime, atime);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setTimes(src, mtime, atime);
   }
 
   @Override // ClientProtocol
   public void createSymlink(String target, String link, FsPermission dirPerms,
       boolean createParent) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO Verify that the link location is in the same NS as the targets
-    final List<RemoteLocation> targetLocations =
-        getLocationsForPath(target, true);
-    final List<RemoteLocation> linkLocations =
-        getLocationsForPath(link, true);
-    RemoteLocation linkLocation = linkLocations.get(0);
-    RemoteMethod method = new RemoteMethod("createSymlink",
-        new Class<?>[] {String.class, String.class, FsPermission.class,
-                        boolean.class},
-        new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
-    rpcClient.invokeSequential(targetLocations, method);
+    clientProto.createSymlink(target, link, dirPerms, createParent);
   }
 
   @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(path, true);
-    RemoteMethod method = new RemoteMethod("getLinkTarget",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (String) rpcClient.invokeSequential(
-        locations, method, String.class, null);
+    return clientProto.getLinkTarget(path);
   }
 
   @Override // Client Protocol
   public void allowSnapshot(String snapshotRoot) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.allowSnapshot(snapshotRoot);
   }
 
   @Override // Client Protocol
   public void disallowSnapshot(String snapshot) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.disallowSnapshot(snapshot);
   }
 
   @Override // ClientProtocol
   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.renameSnapshot(snapshotRoot, snapshotOldName, snapshotNewName);
   }
 
   @Override // Client Protocol
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshottableDirListing();
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshotDiffReport(
+        snapshotRoot, earlierSnapshotName, laterSnapshotName);
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReportListing getSnapshotDiffReportListing(
       String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
       byte[] startPath, int index) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshotDiffReportListing(snapshotRoot,
+        earlierSnapshotName, laterSnapshotName, startPath, index);
   }
 
   @Override // ClientProtocol
   public long addCacheDirective(CacheDirectiveInfo path,
       EnumSet<CacheFlag> flags) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return 0;
+    return clientProto.addCacheDirective(path, flags);
   }
 
   @Override // ClientProtocol
   public void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.modifyCacheDirective(directive, flags);
   }
 
   @Override // ClientProtocol
   public void removeCacheDirective(long id) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.removeCacheDirective(id);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
       long prevId, CacheDirectiveInfo filter) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listCacheDirectives(prevId, filter);
   }
 
   @Override // ClientProtocol
   public void addCachePool(CachePoolInfo info) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.addCachePool(info);
   }
 
   @Override // ClientProtocol
   public void modifyCachePool(CachePoolInfo info) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.modifyCachePool(info);
   }
 
   @Override // ClientProtocol
   public void removeCachePool(String cachePoolName) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.removeCachePool(cachePoolName);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listCachePools(prevKey);
   }
 
   @Override // ClientProtocol
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("modifyAclEntries",
-        new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.modifyAclEntries(src, aclSpec);
   }
 
   @Override // ClienProtocol
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeAclEntries",
-        new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.removeAclEntries(src, aclSpec);
   }
 
   @Override // ClientProtocol
   public void removeDefaultAcl(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeDefaultAcl",
-        new Class<?>[] {String.class}, new RemoteParam());
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeDefaultAcl(src);
   }
 
   @Override // ClientProtocol
   public void removeAcl(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeAcl",
-        new Class<?>[] {String.class}, new RemoteParam());
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeAcl(src);
   }
 
   @Override // ClientProtocol
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod(
-        "setAcl", new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setAcl(src, aclSpec);
   }
 
   @Override // ClientProtocol
   public AclStatus getAclStatus(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getAclStatus",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (AclStatus) rpcClient.invokeSequential(
-        locations, method, AclStatus.class, null);
+    return clientProto.getAclStatus(src);
   }
 
   @Override // ClientProtocol
   public void createEncryptionZone(String src, String keyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("createEncryptionZone",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), keyName);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.createEncryptionZone(src, keyName);
   }
 
   @Override // ClientProtocol
   public EncryptionZone getEZForPath(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getEZForPath",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (EncryptionZone) rpcClient.invokeSequential(
-        locations, method, EncryptionZone.class, null);
+    return clientProto.getEZForPath(src);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listEncryptionZones(prevId);
   }
 
   @Override // ClientProtocol
   public void reencryptEncryptionZone(String zone, ReencryptAction action)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.reencryptEncryptionZone(zone, action);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(
       long prevId) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listReencryptionStatus(prevId);
   }
 
   @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setXAttr",
-        new Class<?>[] {String.class, XAttr.class, EnumSet.class},
-        new RemoteParam(), xAttr, flag);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setXAttr(src, xAttr, flag);
   }
 
-  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getXAttrs",
-        new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
-    return (List<XAttr>) rpcClient.invokeSequential(
-        locations, method, List.class, null);
+    return clientProto.getXAttrs(src, xAttrs);
   }
 
-  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> listXAttrs(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("listXAttrs",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (List<XAttr>) rpcClient.invokeSequential(
-        locations, method, List.class, null);
+    return clientProto.listXAttrs(src);
   }
 
   @Override // ClientProtocol
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeXAttr",
-        new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeXAttr(src, xAttr);
   }
 
   @Override // ClientProtocol
   public void checkAccess(String path, FsAction mode) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(path, true);
-    RemoteMethod method = new RemoteMethod("checkAccess",
-        new Class<?>[] {String.class, FsAction.class},
-        new RemoteParam(), mode);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.checkAccess(path, mode);
   }
 
   @Override // ClientProtocol
   public long getCurrentEditLogTxid() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod(
-        "getCurrentEditLogTxid", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Long> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
-
-    // Return the maximum txid
-    long txid = 0;
-    for (long t : ret.values()) {
-      if (t > txid) {
-        txid = t;
-      }
-    }
-    return txid;
+    return clientProto.getCurrentEditLogTxid();
   }
 
   @Override // ClientProtocol
   public EventBatchList getEditsFromTxid(long txid) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getEditsFromTxid(txid);
   }
 
-  @Override
+  @Override // ClientProtocol
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getDataEncryptionKey();
   }
 
-  @Override
+  @Override // ClientProtocol
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-    return null;
+    return clientProto.createSnapshot(snapshotRoot, snapshotName);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.deleteSnapshot(snapshotRoot, snapshotName);
   }
 
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
-    this.quotaCall.setQuota(path, namespaceQuota, storagespaceQuota, type);
+    clientProto.setQuota(path, namespaceQuota, storagespaceQuota, type);
   }
 
   @Override // ClientProtocol
   public QuotaUsage getQuotaUsage(String path) throws IOException {
-    return this.quotaCall.getQuotaUsage(path);
+    return clientProto.getQuotaUsage(path);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // Block pool id -> blocks
-    Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
-    for (LocatedBlock block : blocks) {
-      String bpId = block.getBlock().getBlockPoolId();
-      List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
-      if (bpBlocks == null) {
-        bpBlocks = new LinkedList<>();
-        blockLocations.put(bpId, bpBlocks);
-      }
-      bpBlocks.add(block);
-    }
-
-    // Invoke each block pool
-    for (Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
-      String bpId = entry.getKey();
-      List<LocatedBlock> bpBlocks = entry.getValue();
-
-      LocatedBlock[] bpBlocksArray =
-          bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
-      RemoteMethod method = new RemoteMethod("reportBadBlocks",
-          new Class<?>[] {LocatedBlock[].class},
-          new Object[] {bpBlocksArray});
-      rpcClient.invokeSingleBlockPool(bpId, method);
-    }
+    clientProto.reportBadBlocks(blocks);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void unsetStoragePolicy(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.unsetStoragePolicy(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getStoragePolicy(path);
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
       throws IOException {
-    return erasureCoding.getErasureCodingPolicies();
+    return clientProto.getErasureCodingPolicies();
   }
 
   @Override // ClientProtocol
   public Map<String, String> getErasureCodingCodecs() throws IOException {
-    return erasureCoding.getErasureCodingCodecs();
+    return clientProto.getErasureCodingCodecs();
   }
 
   @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    return erasureCoding.addErasureCodingPolicies(policies);
+    return clientProto.addErasureCodingPolicies(policies);
   }
 
   @Override // ClientProtocol
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
+    clientProto.removeErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void disableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
+    clientProto.disableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void enableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
+    clientProto.enableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws IOException {
-    return erasureCoding.getErasureCodingPolicy(src);
+    return clientProto.getErasureCodingPolicy(src);
   }
 
   @Override // ClientProtocol
   public void setErasureCodingPolicy(String src, String ecPolicyName)
       throws IOException {
-    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+    clientProto.setErasureCodingPolicy(src, ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void unsetErasureCodingPolicy(String src) throws IOException {
-    erasureCoding.unsetErasureCodingPolicy(src);
+    clientProto.unsetErasureCodingPolicy(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    return erasureCoding.getECBlockGroupStats();
+    return clientProto.getECBlockGroupStats();
   }
 
-  @Override
+  @Override // ClientProtocol
   public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getReplicatedBlockStats();
   }
 
   @Deprecated
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
-        OpenFilesIterator.FILTER_PATH_DEFAULT);
+    return clientProto.listOpenFiles(prevId);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
       EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listOpenFiles(prevId, openFilesTypes, path);
+  }
+
+  @Override // ClientProtocol
+  public void satisfyStoragePolicy(String path) throws IOException {
+    clientProto.satisfyStoragePolicy(path);
   }
 
   @Override // NamenodeProtocol
@@ -2167,6 +1266,11 @@ public class RouterRpcServer extends AbstractService
     return nnProto.isRollingUpgrade();
   }
 
+  @Override // NamenodeProtocol
+  public Long getNextSPSPath() throws IOException {
+    return nnProto.getNextSPSPath();
+  }
+
   /**
    * Locate the location with the matching block pool id.
    *
@@ -2176,7 +1280,7 @@ public class RouterRpcServer extends AbstractService
    * @return Prioritized list of locations in the federated cluster.
    * @throws IOException if the location for this path cannot be determined.
    */
-  private RemoteLocation getLocationForPath(
+  protected RemoteLocation getLocationForPath(
       String path, boolean failIfLocked, String blockPoolId)
           throws IOException {
 
@@ -2276,27 +1380,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
-   * Check if a path should be in all subclusters.
-   *
-   * @param path Path to check.
-   * @return If a path should be in all subclusters.
-   */
-  private boolean isPathAll(final String path) {
-    if (subclusterResolver instanceof MountTableResolver) {
-      try {
-        MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
-        MountTable entry = mountTable.getMountPoint(path);
-        if (entry != null) {
-          return entry.isAll();
-        }
-      } catch (IOException e) {
-        LOG.error("Cannot get mount point", e);
-      }
-    }
-    return false;
-  }
-
-  /**
    * Check if a path is in a read only mount point.
    *
    * @param path Path to check.
@@ -2318,121 +1401,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
-   * Get the modification dates for mount points.
-   *
-   * @param path Name of the path to start checking dates from.
-   * @return Map with the modification dates for all sub-entries.
-   */
-  private Map<String, Long> getMountPointDates(String path) {
-    Map<String, Long> ret = new TreeMap<>();
-    if (subclusterResolver instanceof MountTableResolver) {
-      try {
-        final List<String> children = subclusterResolver.getMountPoints(path);
-        for (String child : children) {
-          Long modTime = getModifiedTime(ret, path, child);
-          ret.put(child, modTime);
-        }
-      } catch (IOException e) {
-        LOG.error("Cannot get mount point", e);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * Get modified time for child. If the child is present in mount table it
-   * will return the modified time. If the child is not present but subdirs of
-   * this child are present then it will return latest modified subdir's time
-   * as modified time of the requested child.
-   * @param ret contains children and modified times.
-   * @param mountTable.
-   * @param path Name of the path to start checking dates from.
-   * @param child child of the requested path.
-   * @return modified time.
-   */
-  private long getModifiedTime(Map<String, Long> ret, String path,
-      String child) {
-    MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
-    String srcPath;
-    if (path.equals(Path.SEPARATOR)) {
-      srcPath = Path.SEPARATOR + child;
-    } else {
-      srcPath = path + Path.SEPARATOR + child;
-    }
-    Long modTime = 0L;
-    try {
-      // Get mount table entry for the srcPath
-      MountTable entry = mountTable.getMountPoint(srcPath);
-      // if srcPath is not in mount table but its subdirs are in mount
-      // table we will display latest modified subdir date/time.
-      if (entry == null) {
-        List<MountTable> entries = mountTable.getMounts(srcPath);
-        for (MountTable eachEntry : entries) {
-          // Get the latest date
-          if (ret.get(child) == null ||
-              ret.get(child) < eachEntry.getDateModified()) {
-            modTime = eachEntry.getDateModified();
-          }
-        }
-      } else {
-        modTime = entry.getDateModified();
-      }
-    } catch (IOException e) {
-      LOG.error("Cannot get mount point", e);
-    }
-    return modTime;
-  }
-
-  /**
-   * Create a new file status for a mount point.
-   *
-   * @param name Name of the mount point.
-   * @param childrenNum Number of children.
-   * @param date Map with the dates.
-   * @return New HDFS file status representing a mount point.
-   */
-  private HdfsFileStatus getMountPointStatus(
-      String name, int childrenNum, long date) {
-    long modTime = date;
-    long accessTime = date;
-    FsPermission permission = FsPermission.getDirDefault();
-    String owner = this.superUser;
-    String group = this.superGroup;
-    try {
-      // TODO support users, it should be the user for the pointed folder
-      UserGroupInformation ugi = getRemoteUser();
-      owner = ugi.getUserName();
-      group = ugi.getPrimaryGroupName();
-    } catch (IOException e) {
-      LOG.error("Cannot get the remote user: {}", e.getMessage());
-    }
-    long inodeId = 0;
-    return new HdfsFileStatus.Builder()
-      .isdir(true)
-      .mtime(modTime)
-      .atime(accessTime)
-      .perm(permission)
-      .owner(owner)
-      .group(group)
-      .symlink(new byte[0])
-      .path(DFSUtil.string2Bytes(name))
-      .fileId(inodeId)
-      .children(childrenNum)
-      .build();
-  }
-
-  /**
-   * Get the name of the method that is calling this function.
-   *
-   * @return Name of the method calling this function.
-   */
-  private static String getMethodName() {
-    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
-    String methodName = stack[3].getMethodName();
-    return methodName;
-  }
-
-  /**
    * Get the user that is invoking this operation.
    *
    * @return Remote user group information.
@@ -2490,16 +1458,4 @@ public class RouterRpcServer extends AbstractService
   public FederationRPCMetrics getRPCMetrics() {
     return this.rpcMonitor.getRPCMetrics();
   }
-
-  @Override
-  public void satisfyStoragePolicy(String path) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
-  @Override
-  public Long getNextSPSPath() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    // not supported
-    return null;
-  }
 }


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


[20/50] [abbrv] hadoop git commit: YARN-8673. [AMRMProxy] More robust responseId resync after an YarnRM master slave switch. Contributed by Botong Huang.

Posted by sh...@apache.org.
YARN-8673. [AMRMProxy] More robust responseId resync after an YarnRM master slave switch. Contributed by Botong Huang.


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

Branch: refs/heads/HDFS-12943
Commit: 8736fc39ac3b3de168d2c216f3d1c0edb48fb3f9
Parents: 65e7469
Author: Giovanni Matteo Fumarola <gi...@apache.com>
Authored: Mon Aug 20 12:22:36 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.com>
Committed: Mon Aug 20 12:22:36 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/AMRMClientUtils.java     |  47 +++++++
 .../hadoop/yarn/server/AMRMClientRelayer.java   | 130 +++++++++++++------
 .../server/uam/UnmanagedApplicationManager.java |   2 +-
 .../yarn/server/TestAMRMClientRelayer.java      |  53 +++++++-
 .../amrmproxy/FederationInterceptor.java        |   6 +-
 .../ApplicationMasterService.java               |   9 +-
 6 files changed, 190 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
index 5d4ab4a6..b8319cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AMRMClientUtils.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -54,6 +55,10 @@ public final class AMRMClientUtils {
   public static final String APP_ALREADY_REGISTERED_MESSAGE =
       "Application Master is already registered : ";
 
+  public static final String EXPECTED_HB_RESPONSEID_MESSAGE =
+      " expect responseId to be ";
+  public static final String RECEIVED_HB_RESPONSEID_MESSAGE = " but get ";
+
   private AMRMClientUtils() {
   }
 
@@ -105,6 +110,48 @@ public final class AMRMClientUtils {
         SaslRpcServer.AuthMethod.TOKEN.toString());
   }
 
+  /**
+   * Generate the exception message when RM receives an AM heartbeat with
+   * invalid responseId.
+   *
+   * @param appAttemptId the app attempt
+   * @param expected the expected responseId value
+   * @param received the received responseId value
+   * @return the assembled exception message
+   */
+  public static String assembleInvalidResponseIdExceptionMessage(
+      ApplicationAttemptId appAttemptId, int expected, int received) {
+    return "Invalid responseId in AllocateRequest from application attempt: "
+        + appAttemptId + EXPECTED_HB_RESPONSEID_MESSAGE + expected
+        + RECEIVED_HB_RESPONSEID_MESSAGE + received;
+  }
+
+  /**
+   * Parse the expected responseId from the exception generated by RM when
+   * processing AM heartbeat.
+   *
+   * @param exceptionMessage the exception message thrown by RM
+   * @return the parsed expected responseId, -1 if failed
+   */
+  public static int parseExpectedResponseIdFromException(
+      String exceptionMessage) {
+    if (exceptionMessage == null) {
+      return -1;
+    }
+    int start = exceptionMessage.indexOf(EXPECTED_HB_RESPONSEID_MESSAGE);
+    int end = exceptionMessage.indexOf(RECEIVED_HB_RESPONSEID_MESSAGE);
+    if (start == -1 || end == -1) {
+      return -1;
+    }
+    start += EXPECTED_HB_RESPONSEID_MESSAGE.length();
+
+    try {
+      return Integer.parseInt(exceptionMessage.substring(start, end));
+    } catch (NumberFormatException ex) {
+      return -1;
+    }
+  }
+
   public static void addToOutstandingSchedulingRequests(
       Collection<SchedulingRequest> requests,
       Map<Set<String>, List<SchedulingRequest>> outstandingSchedRequests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
index 0d1a27e..1e2060c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMRMClientRelayer.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.client.AMRMClientUtils;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.scheduler.ResourceRequestSet;
@@ -111,13 +113,22 @@ public class AMRMClientRelayer extends AbstractService
       new HashMap<>();
   private List<SchedulingRequest> schedulingRequest = new ArrayList<>();
 
+  private ApplicationId appId;
+
+  // Normally -1, otherwise will override responseId with this value in the next
+  // heartbeat
+  private volatile int resetResponseId;
+
   public AMRMClientRelayer() {
     super(AMRMClientRelayer.class.getName());
+    this.resetResponseId = -1;
   }
 
-  public AMRMClientRelayer(ApplicationMasterProtocol rmClient) {
+  public AMRMClientRelayer(ApplicationMasterProtocol rmClient,
+      ApplicationId appId) {
     this();
     this.rmClient = rmClient;
+    this.appId = appId;
   }
 
   @Override
@@ -167,59 +178,63 @@ public class AMRMClientRelayer extends AbstractService
     try {
       return this.rmClient.finishApplicationMaster(request);
     } catch (ApplicationMasterNotRegisteredException e) {
-      LOG.warn("Out of sync with ResourceManager, hence resyncing.");
+      LOG.warn("Out of sync with RM for " + this.appId + ", hence resyncing.");
       // re register with RM
       registerApplicationMaster(this.amRegistrationRequest);
       return finishApplicationMaster(request);
     }
   }
 
+  private void addNewAllocateRequest(AllocateRequest allocateRequest)
+      throws YarnException {
+    // update the data structures first
+    addNewAsks(allocateRequest.getAskList());
+
+    if (allocateRequest.getReleaseList() != null) {
+      this.remotePendingRelease.addAll(allocateRequest.getReleaseList());
+      this.release.addAll(allocateRequest.getReleaseList());
+    }
+
+    if (allocateRequest.getResourceBlacklistRequest() != null) {
+      if (allocateRequest.getResourceBlacklistRequest()
+          .getBlacklistAdditions() != null) {
+        this.remoteBlacklistedNodes.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistAdditions());
+        this.blacklistAdditions.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistAdditions());
+      }
+      if (allocateRequest.getResourceBlacklistRequest()
+          .getBlacklistRemovals() != null) {
+        this.remoteBlacklistedNodes.removeAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistRemovals());
+        this.blacklistRemovals.addAll(allocateRequest
+            .getResourceBlacklistRequest().getBlacklistRemovals());
+      }
+    }
+
+    if (allocateRequest.getUpdateRequests() != null) {
+      for (UpdateContainerRequest update : allocateRequest
+          .getUpdateRequests()) {
+        this.remotePendingChange.put(update.getContainerId(), update);
+        this.change.put(update.getContainerId(), update);
+      }
+    }
+
+    if (allocateRequest.getSchedulingRequests() != null) {
+      AMRMClientUtils.addToOutstandingSchedulingRequests(
+          allocateRequest.getSchedulingRequests(),
+          this.remotePendingSchedRequest);
+      this.schedulingRequest.addAll(allocateRequest.getSchedulingRequests());
+    }
+  }
+
   @Override
   public AllocateResponse allocate(AllocateRequest allocateRequest)
       throws YarnException, IOException {
     AllocateResponse allocateResponse = null;
     try {
       synchronized (this) {
-        // update the data structures first
-        addNewAsks(allocateRequest.getAskList());
-
-        if (allocateRequest.getReleaseList() != null) {
-          this.remotePendingRelease.addAll(allocateRequest.getReleaseList());
-          this.release.addAll(allocateRequest.getReleaseList());
-        }
-
-        if (allocateRequest.getResourceBlacklistRequest() != null) {
-          if (allocateRequest.getResourceBlacklistRequest()
-              .getBlacklistAdditions() != null) {
-            this.remoteBlacklistedNodes.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistAdditions());
-            this.blacklistAdditions.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistAdditions());
-          }
-          if (allocateRequest.getResourceBlacklistRequest()
-              .getBlacklistRemovals() != null) {
-            this.remoteBlacklistedNodes.removeAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistRemovals());
-            this.blacklistRemovals.addAll(allocateRequest
-                .getResourceBlacklistRequest().getBlacklistRemovals());
-          }
-        }
-
-        if (allocateRequest.getUpdateRequests() != null) {
-          for (UpdateContainerRequest update : allocateRequest
-              .getUpdateRequests()) {
-            this.remotePendingChange.put(update.getContainerId(), update);
-            this.change.put(update.getContainerId(), update);
-          }
-        }
-
-        if (allocateRequest.getSchedulingRequests() != null) {
-          AMRMClientUtils.addToOutstandingSchedulingRequests(
-              allocateRequest.getSchedulingRequests(),
-              this.remotePendingSchedRequest);
-          this.schedulingRequest
-              .addAll(allocateRequest.getSchedulingRequests());
-        }
+        addNewAllocateRequest(allocateRequest);
 
         ArrayList<ResourceRequest> askList = new ArrayList<>(ask.size());
         for (ResourceRequest r : ask) {
@@ -238,13 +253,23 @@ public class AMRMClientRelayer extends AbstractService
             .updateRequests(new ArrayList<>(this.change.values()))
             .schedulingRequests(new ArrayList<>(this.schedulingRequest))
             .build();
+
+        if (this.resetResponseId != -1) {
+          LOG.info("Override allocate responseId from "
+              + allocateRequest.getResponseId() + " to " + this.resetResponseId
+              + " for " + this.appId);
+          allocateRequest.setResponseId(this.resetResponseId);
+        }
       }
 
       // Do the actual allocate call
       try {
         allocateResponse = this.rmClient.allocate(allocateRequest);
+
+        // Heartbeat succeeded, wipe out responseId overriding
+        this.resetResponseId = -1;
       } catch (ApplicationMasterNotRegisteredException e) {
-        LOG.warn("ApplicationMaster is out of sync with ResourceManager,"
+        LOG.warn("ApplicationMaster is out of sync with RM for " + this.appId
             + " hence resyncing.");
 
         synchronized (this) {
@@ -269,6 +294,25 @@ public class AMRMClientRelayer extends AbstractService
         // Reset responseId after re-register
         allocateRequest.setResponseId(0);
         return allocate(allocateRequest);
+      } catch (Throwable t) {
+
+        // If RM is complaining about responseId out of sync, force reset next
+        // time
+        if (t instanceof InvalidApplicationMasterRequestException) {
+          int responseId = AMRMClientUtils
+              .parseExpectedResponseIdFromException(t.getMessage());
+          if (responseId != -1) {
+            this.resetResponseId = responseId;
+            LOG.info("ResponseId out of sync with RM, expect " + responseId
+                + " but " + allocateRequest.getResponseId() + " used by "
+                + this.appId + ". Will override in the next allocate.");
+          } else {
+            LOG.warn("Failed to parse expected responseId out of exception for "
+                + this.appId);
+          }
+        }
+
+        throw t;
       }
 
       synchronized (this) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
index 856a818..abdec19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
@@ -193,7 +193,7 @@ public class UnmanagedApplicationManager {
         this.applicationId.toString(), UserGroupInformation.getCurrentUser());
     this.rmProxyRelayer =
         new AMRMClientRelayer(createRMProxy(ApplicationMasterProtocol.class,
-            this.conf, this.userUgi, amrmToken));
+            this.conf, this.userUgi, amrmToken), this.applicationId);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
index 22bb1f9..4c84f0b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/TestAMRMClientRelayer.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.client.AMRMClientUtils;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.scheduler.ResourceRequestSet;
 import org.apache.hadoop.yarn.util.Records;
@@ -62,6 +64,7 @@ public class TestAMRMClientRelayer {
     // Whether this mockRM will throw failover exception upon next heartbeat
     // from AM
     private boolean failover = false;
+    private int responseIdReset = -1;
     private List<ResourceRequest> lastAsk;
     private List<ContainerId> lastRelease;
     private List<String> lastBlacklistAdditions;
@@ -92,26 +95,40 @@ public class TestAMRMClientRelayer {
         this.failover = false;
         throw new ApplicationMasterNotRegisteredException("Mock RM restarted");
       }
+      if (this.responseIdReset != -1) {
+        String errorMessage =
+            AMRMClientUtils.assembleInvalidResponseIdExceptionMessage(null,
+                this.responseIdReset, request.getResponseId());
+        this.responseIdReset = -1;
+        throw new InvalidApplicationMasterRequestException(errorMessage);
+      }
+
       this.lastAsk = request.getAskList();
       this.lastRelease = request.getReleaseList();
       this.lastBlacklistAdditions =
           request.getResourceBlacklistRequest().getBlacklistAdditions();
       this.lastBlacklistRemovals =
           request.getResourceBlacklistRequest().getBlacklistRemovals();
-      return AllocateResponse.newInstance(0, null, null,
-          new ArrayList<NodeReport>(), Resource.newInstance(0, 0), null, 0,
-          null, null);
+      return AllocateResponse.newInstance(request.getResponseId() + 1, null,
+          null, new ArrayList<NodeReport>(), Resource.newInstance(0, 0), null,
+          0, null, null);
     }
 
     public void setFailoverFlag() {
       this.failover = true;
     }
+
+    public void setResponseIdReset(int expectedResponseId) {
+      this.responseIdReset = expectedResponseId;
+    }
   }
 
   private Configuration conf;
   private MockApplicationMasterService mockAMS;
   private AMRMClientRelayer relayer;
 
+  private int responseId = 0;
+
   // Buffer of asks that will be sent to RM in the next AM heartbeat
   private List<ResourceRequest> asks = new ArrayList<>();
   private List<ContainerId> releases = new ArrayList<>();
@@ -123,7 +140,7 @@ public class TestAMRMClientRelayer {
     this.conf = new Configuration();
 
     this.mockAMS = new MockApplicationMasterService();
-    this.relayer = new AMRMClientRelayer(this.mockAMS);
+    this.relayer = new AMRMClientRelayer(this.mockAMS, null);
 
     this.relayer.init(conf);
     this.relayer.start();
@@ -150,7 +167,7 @@ public class TestAMRMClientRelayer {
   private AllocateRequest getAllocateRequest() {
     // Need to create a new one every time because rather than directly
     // referring the lists, the protobuf impl makes a copy of the lists
-    return AllocateRequest.newInstance(0, 0, asks, releases,
+    return AllocateRequest.newInstance(responseId, 0, asks, releases,
         ResourceBlacklistRequest.newInstance(blacklistAdditions,
             blacklistRemoval));
   }
@@ -272,4 +289,30 @@ public class TestAMRMClientRelayer {
     clearAllocateRequestLists();
   }
 
+  @Test
+  public void testResponseIdResync() throws YarnException, IOException {
+    this.responseId = 10;
+
+    AllocateResponse response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(this.responseId + 1, response.getResponseId());
+
+    int expected = 5;
+    this.mockAMS.setResponseIdReset(expected);
+
+    try {
+      this.relayer.allocate(getAllocateRequest());
+      Assert.fail("Expecting exception from RM");
+    } catch (InvalidApplicationMasterRequestException e) {
+      // Expected exception
+    }
+
+    // Verify that the responseId is overridden
+    response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(expected + 1, response.getResponseId());
+
+    // Verify it is no longer overriden
+    this.responseId = response.getResponseId();
+    response = this.relayer.allocate(getAllocateRequest());
+    Assert.assertEquals(this.responseId + 1, response.getResponseId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index 645e47e..65a2277 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -249,8 +249,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
-    this.homeRMRelayer = new AMRMClientRelayer(createHomeRMProxy(appContext,
-        ApplicationMasterProtocol.class, this.appOwner));
+    this.homeRMRelayer = new AMRMClientRelayer(
+        createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
+            this.appOwner),
+        getApplicationContext().getApplicationAttemptId().getApplicationId());
 
     this.federationFacade = FederationStateStoreFacade.getInstance();
     this.subClusterResolver = this.federationFacade.getSubClusterResolver();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8736fc39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 7dac2cd..6fe0aa9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -420,12 +420,9 @@ public class ApplicationMasterService extends AbstractService implements
         // heartbeat one step old, simply return lastReponse
         return lastResponse;
       } else if (request.getResponseId() != lastResponse.getResponseId()) {
-        String message =
-            "Invalid responseId in AllocateRequest from application attempt: "
-                + appAttemptId + ", expect responseId to be "
-                + lastResponse.getResponseId() + ", but get "
-                + request.getResponseId();
-        throw new InvalidApplicationMasterRequestException(message);
+        throw new InvalidApplicationMasterRequestException(AMRMClientUtils
+            .assembleInvalidResponseIdExceptionMessage(appAttemptId,
+                lastResponse.getResponseId(), request.getResponseId()));
       }
 
       AllocateResponse response =


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


[38/50] [abbrv] hadoop git commit: HDDS-328. Support export and import of the KeyValueContainer. Contributed by Elek Marton.

Posted by sh...@apache.org.
HDDS-328. Support export and import of the KeyValueContainer. Contributed by Elek Marton.


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

Branch: refs/heads/HDFS-12943
Commit: ca29fb754e8a162edba380a5f1deb48699e14d8b
Parents: 585ebd8
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Thu Aug 23 11:30:28 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Thu Aug 23 11:30:28 2018 -0700

----------------------------------------------------------------------
 .../common/impl/ContainerDataYaml.java          |  94 ++++---
 .../container/common/interfaces/Container.java  |  33 ++-
 .../common/interfaces/ContainerPacker.java      |  58 +++++
 .../container/keyvalue/KeyValueContainer.java   | 128 ++++++++--
 .../container/keyvalue/KeyValueHandler.java     |  19 +-
 .../container/keyvalue/TarContainerPacker.java  | 249 +++++++++++++++++++
 .../keyvalue/helpers/KeyValueContainerUtil.java |  22 +-
 .../keyvalue/TestKeyValueContainer.java         |  95 ++++++-
 .../keyvalue/TestTarContainerPacker.java        | 231 +++++++++++++++++
 9 files changed, 849 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
index aed75d3..ec6d642 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerDataYaml.java
@@ -18,31 +18,34 @@
 
 package org.apache.hadoop.ozone.container.common.impl;
 
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerType;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
-
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.yaml.snakeyaml.Yaml;
-
 import java.beans.IntrospectionException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Writer;
+import java.io.ByteArrayInputStream;
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStreamWriter;
-import java.io.File;
+import java.io.Writer;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.Map;
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerType;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+
+import com.google.common.base.Preconditions;
+import static org.apache.hadoop.ozone.container.keyvalue
+    .KeyValueContainerData.KEYVALUE_YAML_TAG;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.AbstractConstruct;
 import org.yaml.snakeyaml.constructor.Constructor;
 import org.yaml.snakeyaml.introspector.BeanAccess;
@@ -54,9 +57,6 @@ import org.yaml.snakeyaml.nodes.ScalarNode;
 import org.yaml.snakeyaml.nodes.Tag;
 import org.yaml.snakeyaml.representer.Representer;
 
-import static org.apache.hadoop.ozone.container.keyvalue
-    .KeyValueContainerData.KEYVALUE_YAML_TAG;
-
 /**
  * Class for creating and reading .container files.
  */
@@ -106,36 +106,52 @@ public final class ContainerDataYaml {
   /**
    * Read the yaml file, and return containerData.
    *
-   * @param containerFile
    * @throws IOException
    */
   public static ContainerData readContainerFile(File containerFile)
       throws IOException {
     Preconditions.checkNotNull(containerFile, "containerFile cannot be null");
+    try (FileInputStream inputFileStream = new FileInputStream(containerFile)) {
+      return readContainer(inputFileStream);
+    }
+
+  }
+
+  /**
+   * Read the yaml file content, and return containerData.
+   *
+   * @throws IOException
+   */
+  public static ContainerData readContainer(byte[] containerFileContent)
+      throws IOException {
+    return readContainer(
+        new ByteArrayInputStream(containerFileContent));
+  }
+
+  /**
+   * Read the yaml content, and return containerData.
+   *
+   * @throws IOException
+   */
+  public static ContainerData readContainer(InputStream input)
+      throws IOException {
 
-    InputStream input = null;
     ContainerData containerData;
-    try {
-      PropertyUtils propertyUtils = new PropertyUtils();
-      propertyUtils.setBeanAccess(BeanAccess.FIELD);
-      propertyUtils.setAllowReadOnlyProperties(true);
+    PropertyUtils propertyUtils = new PropertyUtils();
+    propertyUtils.setBeanAccess(BeanAccess.FIELD);
+    propertyUtils.setAllowReadOnlyProperties(true);
 
-      Representer representer = new ContainerDataRepresenter();
-      representer.setPropertyUtils(propertyUtils);
+    Representer representer = new ContainerDataRepresenter();
+    representer.setPropertyUtils(propertyUtils);
 
-      Constructor containerDataConstructor = new ContainerDataConstructor();
+    Constructor containerDataConstructor = new ContainerDataConstructor();
 
-      Yaml yaml = new Yaml(containerDataConstructor, representer);
-      yaml.setBeanAccess(BeanAccess.FIELD);
+    Yaml yaml = new Yaml(containerDataConstructor, representer);
+    yaml.setBeanAccess(BeanAccess.FIELD);
+
+    containerData = (ContainerData)
+        yaml.load(input);
 
-      input = new FileInputStream(containerFile);
-      containerData = (ContainerData)
-          yaml.load(input);
-    } finally {
-      if (input!= null) {
-        input.close();
-      }
-    }
     return containerData;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index 7f706b5..9380f0c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -18,26 +18,27 @@
 
 package org.apache.hadoop.ozone.container.common.interfaces;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .StorageContainerException;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.scm.container.common.helpers.
-    StorageContainerException;
 
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Map;
-
-
 /**
  * Interface for Container Operations.
  */
-public interface Container extends RwLock {
+public interface Container<CONTAINERDATA extends ContainerData> extends RwLock {
 
   /**
    * Creates a container.
@@ -71,7 +72,7 @@ public interface Container extends RwLock {
    * @return ContainerData - Container Data.
    * @throws StorageContainerException
    */
-  ContainerData getContainerData();
+  CONTAINERDATA getContainerData();
 
   /**
    * Get the Container Lifecycle state.
@@ -113,6 +114,20 @@ public interface Container extends RwLock {
   BlockIterator blockIterator() throws IOException;
 
   /**
+   * Import the container from an external archive.
+   */
+  void importContainerData(InputStream stream,
+      ContainerPacker<CONTAINERDATA> packer) throws IOException;
+
+  /**
+   * Export all the data of the container to one output archive with the help
+   * of the packer.
+   *
+   */
+  void exportContainerData(OutputStream stream,
+      ContainerPacker<CONTAINERDATA> packer) throws IOException;
+
+  /**
    * Returns containerReport for the container.
    */
   StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
new file mode 100644
index 0000000..8308c23
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerPacker.java
@@ -0,0 +1,58 @@
+/**
+ * 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.container.common.interfaces;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+
+/**
+ * Service to pack/unpack ContainerData container data to/from a single byte
+ * stream.
+ */
+public interface ContainerPacker<CONTAINERDATA extends ContainerData> {
+
+  /**
+   * Extract the container data to the path defined by the container.
+   * <p>
+   * This doesn't contain the extraction of the container descriptor file.
+   *
+   * @return the byte content of the descriptor (which won't be written to a
+   * file but returned).
+   */
+  byte[] unpackContainerData(Container<CONTAINERDATA> container,
+      InputStream inputStream)
+      throws IOException;
+
+  /**
+   * Compress all the container data (chunk data, metadata db AND container
+   * descriptor) to one single archive.
+   */
+  void pack(Container<CONTAINERDATA> container, OutputStream destination)
+      throws IOException;
+
+  /**
+   * Read the descriptor from the finished archive to get the data before
+   * importing the container.
+   */
+  byte[] unpackContainerDescriptor(InputStream inputStream)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 0ea748a..8108a11 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -18,9 +18,15 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
-import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.StandardCopyOption;
+import java.util.Map;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileUtil;
@@ -37,32 +43,27 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
-import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
-import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
+import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.keyvalue.helpers
     .KeyValueContainerLocationUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.utils.MetadataStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_ALREADY_EXISTS;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.CONTAINER_INTERNAL_ERROR;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.CONTAINER_FILES_CREATE_ERROR;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .Result.CONTAINER_INTERNAL_ERROR;
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.DISK_OUT_OF_SPACE;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.ERROR_IN_COMPACT_DB;
@@ -70,11 +71,13 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.INVALID_CONTAINER_STATE;
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.UNSUPPORTED_REQUEST;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class to perform KeyValue Container operations.
  */
-public class KeyValueContainer implements Container {
+public class KeyValueContainer implements Container<KeyValueContainerData> {
 
   private static final Logger LOG = LoggerFactory.getLogger(Container.class);
 
@@ -167,6 +170,34 @@ public class KeyValueContainer implements Container {
   }
 
   /**
+   * Set all of the path realted container data fields based on the name
+   * conventions.
+   *
+   * @param scmId
+   * @param containerVolume
+   * @param hddsVolumeDir
+   */
+  public void populatePathFields(String scmId,
+      HddsVolume containerVolume, String hddsVolumeDir) {
+
+    long containerId = containerData.getContainerID();
+
+    File containerMetaDataPath = KeyValueContainerLocationUtil
+        .getContainerMetaDataPath(hddsVolumeDir, scmId, containerId);
+
+    File chunksPath = KeyValueContainerLocationUtil.getChunksLocationPath(
+        hddsVolumeDir, scmId, containerId);
+    File dbFile = KeyValueContainerLocationUtil.getContainerDBFile(
+        containerMetaDataPath, containerId);
+
+    //Set containerData for the KeyValueContainer.
+    containerData.setMetadataPath(containerMetaDataPath.getPath());
+    containerData.setChunksPath(chunksPath.getPath());
+    containerData.setDbFile(dbFile);
+    containerData.setVolume(containerVolume);
+  }
+
+  /**
    * Writes to .container file.
    *
    * @param containerFile container file name
@@ -334,6 +365,75 @@ public class KeyValueContainer implements Container {
         containerData.getContainerPath()));
   }
 
+  @Override
+  public void importContainerData(InputStream input,
+      ContainerPacker<KeyValueContainerData> packer) throws IOException {
+    writeLock();
+    try {
+      if (getContainerFile().exists()) {
+        String errorMessage = String.format(
+            "Can't import container (cid=%d) data to a specific location"
+                + " as the container descriptor (%s) has already been exist.",
+            getContainerData().getContainerID(),
+            getContainerFile().getAbsolutePath());
+        throw new IOException(errorMessage);
+      }
+      //copy the values from the input stream to the final destination
+      // directory.
+      byte[] descriptorContent = packer.unpackContainerData(this, input);
+
+      Preconditions.checkNotNull(descriptorContent,
+          "Container descriptor is missing from the container archive: "
+              + getContainerData().getContainerID());
+
+      //now, we have extracted the container descriptor from the previous
+      //datanode. We can load it and upload it with the current data
+      // (original metadata + current filepath fields)
+      KeyValueContainerData originalContainerData =
+          (KeyValueContainerData) ContainerDataYaml
+              .readContainer(descriptorContent);
+
+
+      containerData.setState(originalContainerData.getState());
+      containerData
+          .setContainerDBType(originalContainerData.getContainerDBType());
+      containerData.setBytesUsed(originalContainerData.getBytesUsed());
+
+      //rewriting the yaml file with new checksum calculation.
+      update(originalContainerData.getMetadata(), true);
+
+      //fill in memory stat counter (keycount, byte usage)
+      KeyValueContainerUtil.parseKVContainerData(containerData, config);
+
+    } catch (Exception ex) {
+      //delete all the temporary data in case of any exception.
+      try {
+        FileUtils.deleteDirectory(new File(containerData.getMetadataPath()));
+        FileUtils.deleteDirectory(new File(containerData.getChunksPath()));
+        FileUtils.deleteDirectory(getContainerFile());
+      } catch (Exception deleteex) {
+        LOG.error(
+            "Can not cleanup destination directories after a container import"
+                + " error (cid" +
+                containerData.getContainerID() + ")", deleteex);
+      }
+      throw ex;
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  @Override
+  public void exportContainerData(OutputStream destination,
+      ContainerPacker<KeyValueContainerData> packer) throws IOException {
+    if (getContainerData().getState() != ContainerLifeCycleState.CLOSED) {
+      throw new IllegalStateException(
+          "Only closed containers could be exported: ContainerId="
+              + getContainerData().getContainerID());
+    }
+    packer.pack(this, destination);
+  }
+
   /**
    * Acquire read lock.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 9ddb474..29c359e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
 import org.apache.hadoop.ozone.container.common.impl.OpenContainerBlockMap;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerUtil;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.SmallFileUtils;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
@@ -162,7 +163,8 @@ public class KeyValueHandler extends Handler {
     return volumeChoosingPolicy;
   }
   /**
-   * Returns OpenContainerBlockMap instance
+   * Returns OpenContainerBlockMap instance.
+   *
    * @return OpenContainerBlockMap
    */
   public OpenContainerBlockMap getOpenContainerBlockMap() {
@@ -269,6 +271,19 @@ public class KeyValueHandler extends Handler {
     return ContainerUtils.getSuccessResponse(request);
   }
 
+  public void populateContainerPathFields(KeyValueContainer container,
+      long maxSize) throws IOException {
+    volumeSet.acquireLock();
+    try {
+      HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
+          .getVolumesList(), maxSize);
+      String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
+      container.populatePathFields(scmID, containerVolume, hddsVolumeDir);
+    } finally {
+      volumeSet.releaseLock();
+    }
+  }
+
   /**
    * Handles Read Container Request. Returns the ContainerData as response.
    */
@@ -322,7 +337,7 @@ public class KeyValueHandler extends Handler {
    * Open containers cannot be deleted.
    * Holds writeLock on ContainerSet till the container is removed from
    * containerMap. On disk deletion of container files will happen
-   * asynchornously without the lock.
+   * asynchronously without the lock.
    */
   ContainerCommandResponseProto handleDeleteContainer(
       ContainerCommandRequestProto request, KeyValueContainer kvContainer) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
new file mode 100644
index 0000000..13689a7
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
@@ -0,0 +1,249 @@
+/**
+ * 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.container.keyvalue;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.compress.archivers.ArchiveEntry;
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorInputStream;
+import org.apache.commons.compress.compressors.CompressorOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.io.IOUtils;
+
+/**
+ * Compress/uncompress KeyValueContainer data to a tar.gz archive.
+ */
+public class TarContainerPacker
+    implements ContainerPacker<KeyValueContainerData> {
+
+  private static final String CHUNKS_DIR_NAME = OzoneConsts.STORAGE_DIR_CHUNKS;
+
+  private static final String DB_DIR_NAME = "db";
+
+  private static final String CONTAINER_FILE_NAME = "container.yaml";
+
+
+
+  /**
+   * Given an input stream (tar file) extract the data to the specified
+   * directories.
+   *
+   * @param container container which defines the destination structure.
+   * @param inputStream the input stream.
+   * @throws IOException
+   */
+  @Override
+  public byte[] unpackContainerData(Container<KeyValueContainerData> container,
+      InputStream inputStream)
+      throws IOException {
+    byte[] descriptorFileContent = null;
+    try {
+      KeyValueContainerData containerData = container.getContainerData();
+      CompressorInputStream compressorInputStream =
+          new CompressorStreamFactory()
+              .createCompressorInputStream(CompressorStreamFactory.GZIP,
+                  inputStream);
+
+      TarArchiveInputStream tarInput =
+          new TarArchiveInputStream(compressorInputStream);
+
+      TarArchiveEntry entry = tarInput.getNextTarEntry();
+      while (entry != null) {
+        String name = entry.getName();
+        if (name.startsWith(DB_DIR_NAME + "/")) {
+          Path destinationPath = containerData.getDbFile().toPath()
+              .resolve(name.substring(DB_DIR_NAME.length() + 1));
+          extractEntry(tarInput, entry.getSize(), destinationPath);
+        } else if (name.startsWith(CHUNKS_DIR_NAME + "/")) {
+          Path destinationPath = Paths.get(containerData.getChunksPath())
+              .resolve(name.substring(CHUNKS_DIR_NAME.length() + 1));
+          extractEntry(tarInput, entry.getSize(), destinationPath);
+        } else if (name.equals(CONTAINER_FILE_NAME)) {
+          //Don't do anything. Container file should be unpacked in a
+          //separated step by unpackContainerDescriptor call.
+          descriptorFileContent = readEntry(tarInput, entry);
+        } else {
+          throw new IllegalArgumentException(
+              "Unknown entry in the tar file: " + "" + name);
+        }
+        entry = tarInput.getNextTarEntry();
+      }
+      return descriptorFileContent;
+
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't uncompress the given container: " + container
+              .getContainerData().getContainerID(),
+          e);
+    }
+  }
+
+  private void extractEntry(TarArchiveInputStream tarInput, long size,
+      Path path) throws IOException {
+    Preconditions.checkNotNull(path, "Path element should not be null");
+    Path parent = Preconditions.checkNotNull(path.getParent(),
+        "Path element should have a parent directory");
+    Files.createDirectories(parent);
+    try (BufferedOutputStream bos = new BufferedOutputStream(
+        new FileOutputStream(path.toAbsolutePath().toString()))) {
+      int bufferSize = 1024;
+      byte[] buffer = new byte[bufferSize + 1];
+      long remaining = size;
+      while (remaining > 0) {
+        int read =
+            tarInput.read(buffer, 0, (int) Math.min(remaining, bufferSize));
+        if (read >= 0) {
+          remaining -= read;
+          bos.write(buffer, 0, read);
+        } else {
+          remaining = 0;
+        }
+      }
+    }
+
+  }
+
+  /**
+   * Given a containerData include all the required container data/metadata
+   * in a tar file.
+   *
+   * @param container Container to archive (data + metadata).
+   * @param destination   Destination tar file/stream.
+   * @throws IOException
+   */
+  @Override
+  public void pack(Container<KeyValueContainerData> container,
+      OutputStream destination)
+      throws IOException {
+
+    KeyValueContainerData containerData = container.getContainerData();
+
+    try (CompressorOutputStream gzippedOut = new CompressorStreamFactory()
+          .createCompressorOutputStream(CompressorStreamFactory.GZIP,
+              destination)) {
+
+      try (ArchiveOutputStream archiveOutputStream = new TarArchiveOutputStream(
+          gzippedOut)) {
+
+        includePath(containerData.getDbFile().toString(), DB_DIR_NAME,
+            archiveOutputStream);
+
+        includePath(containerData.getChunksPath(), CHUNKS_DIR_NAME,
+            archiveOutputStream);
+
+        includeFile(container.getContainerFile(),
+            CONTAINER_FILE_NAME,
+            archiveOutputStream);
+      }
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't compress the container: " + containerData.getContainerID(),
+          e);
+    }
+
+  }
+
+  @Override
+  public byte[] unpackContainerDescriptor(InputStream inputStream)
+      throws IOException {
+    try {
+      CompressorInputStream compressorInputStream =
+          new CompressorStreamFactory()
+              .createCompressorInputStream(CompressorStreamFactory.GZIP,
+                  inputStream);
+
+      TarArchiveInputStream tarInput =
+          new TarArchiveInputStream(compressorInputStream);
+
+      TarArchiveEntry entry = tarInput.getNextTarEntry();
+      while (entry != null) {
+        String name = entry.getName();
+        if (name.equals(CONTAINER_FILE_NAME)) {
+          return readEntry(tarInput, entry);
+        }
+        entry = tarInput.getNextTarEntry();
+      }
+
+    } catch (CompressorException e) {
+      throw new IOException(
+          "Can't read the container descriptor from the container archive",
+          e);
+    }
+    throw new IOException(
+        "Container descriptor is missing from the container archive.");
+  }
+
+  private byte[] readEntry(TarArchiveInputStream tarInput,
+      TarArchiveEntry entry) throws IOException {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    int bufferSize = 1024;
+    byte[] buffer = new byte[bufferSize + 1];
+    long remaining = entry.getSize();
+    while (remaining > 0) {
+      int read =
+          tarInput.read(buffer, 0, (int) Math.min(remaining, bufferSize));
+      remaining -= read;
+      bos.write(buffer, 0, read);
+    }
+    return bos.toByteArray();
+  }
+
+  private void includePath(String containerPath, String subdir,
+      ArchiveOutputStream archiveOutputStream) throws IOException {
+
+    for (Path path : Files.list(Paths.get(containerPath))
+        .collect(Collectors.toList())) {
+
+      includeFile(path.toFile(), subdir + "/" + path.getFileName(),
+          archiveOutputStream);
+    }
+  }
+
+  private void includeFile(File file, String entryName,
+      ArchiveOutputStream archiveOutputStream) throws IOException {
+    ArchiveEntry archiveEntry =
+        archiveOutputStream.createArchiveEntry(file, entryName);
+    archiveOutputStream.putArchiveEntry(archiveEntry);
+    try (FileInputStream fis = new FileInputStream(file)) {
+      IOUtils.copy(fis, archiveOutputStream);
+    }
+    archiveOutputStream.closeArchiveEntry();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 2352cf6..ed4536f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.ozone.container.keyvalue.helpers;
 
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.FileUtils;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
@@ -32,16 +36,12 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.apache.hadoop.utils.MetadataStoreBuilder;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Class which defines utility methods for KeyValueContainer.
  */
@@ -157,7 +157,7 @@ public final class KeyValueContainerUtil {
    * @throws IOException
    */
   public static void parseKVContainerData(KeyValueContainerData kvContainerData,
-      OzoneConfiguration config) throws IOException {
+      Configuration config) throws IOException {
 
     long containerID = kvContainerData.getContainerID();
     File metadataPath = new File(kvContainerData.getMetadataPath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
index 6ff2eca..7359868 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueContainer.java
@@ -23,7 +23,8 @@ import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 
-
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@@ -37,6 +38,8 @@ import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.utils.MetadataStore;
+
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -46,6 +49,8 @@ import org.mockito.Mockito;
 
 import java.io.File;
 
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -74,7 +79,6 @@ public class TestKeyValueContainer {
   private String scmId = UUID.randomUUID().toString();
   private VolumeSet volumeSet;
   private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
-  private long containerID = 1L;
   private KeyValueContainerData keyValueContainerData;
   private KeyValueContainer keyValueContainer;
 
@@ -141,13 +145,14 @@ public class TestKeyValueContainer {
 
   }
 
+  @SuppressWarnings("RedundantCast")
   @Test
   public void testCreateContainer() throws Exception {
 
     // Create Container.
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     String containerMetaDataPath = keyValueContainerData
@@ -167,6 +172,86 @@ public class TestKeyValueContainer {
   }
 
   @Test
+  public void testContainerImportExport() throws Exception {
+
+    long containerId = keyValueContainer.getContainerData().getContainerID();
+    // Create Container.
+    keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
+
+
+    keyValueContainerData = keyValueContainer
+        .getContainerData();
+
+    keyValueContainerData.setState(ContainerLifeCycleState.CLOSED);
+
+    int numberOfKeysToWrite = 12;
+    //write one few keys to check the key count after import
+    MetadataStore metadataStore = KeyUtils.getDB(keyValueContainerData, conf);
+    for (int i = 0; i < numberOfKeysToWrite; i++) {
+      metadataStore.put(("test" + i).getBytes(), "test".getBytes());
+    }
+    metadataStore.close();
+
+    Map<String, String> metadata = new HashMap<>();
+    metadata.put("key1", "value1");
+    keyValueContainer.update(metadata, true);
+
+    //destination path
+    File folderToExport = folder.newFile("exported.tar.gz");
+
+    TarContainerPacker packer = new TarContainerPacker();
+
+    //export the container
+    try (FileOutputStream fos = new FileOutputStream(folderToExport)) {
+      keyValueContainer
+          .exportContainerData(fos, packer);
+    }
+
+    //delete the original one
+    keyValueContainer.delete(true);
+
+    //create a new one
+    KeyValueContainerData containerData =
+        new KeyValueContainerData(containerId, 1,
+            keyValueContainerData.getMaxSizeGB());
+    KeyValueContainer container = new KeyValueContainer(containerData, conf);
+
+    HddsVolume containerVolume = volumeChoosingPolicy.chooseVolume(volumeSet
+        .getVolumesList(), 1);
+    String hddsVolumeDir = containerVolume.getHddsRootDir().toString();
+
+    container.populatePathFields(scmId, containerVolume, hddsVolumeDir);
+    try (FileInputStream fis = new FileInputStream(folderToExport)) {
+      container.importContainerData(fis, packer);
+    }
+
+    Assert.assertEquals("value1", containerData.getMetadata().get("key1"));
+    Assert.assertEquals(keyValueContainerData.getContainerDBType(),
+        containerData.getContainerDBType());
+    Assert.assertEquals(keyValueContainerData.getState(),
+        containerData.getState());
+    Assert.assertEquals(numberOfKeysToWrite,
+        containerData.getKeyCount());
+    Assert.assertEquals(keyValueContainerData.getLayOutVersion(),
+        containerData.getLayOutVersion());
+    Assert.assertEquals(keyValueContainerData.getMaxSizeGB(),
+        containerData.getMaxSizeGB());
+    Assert.assertEquals(keyValueContainerData.getBytesUsed(),
+        containerData.getBytesUsed());
+
+    //Can't overwrite existing container
+    try {
+      try (FileInputStream fis = new FileInputStream(folderToExport)) {
+        container.importContainerData(fis, packer);
+      }
+      fail("Container is imported twice. Previous files are overwritten");
+    } catch (Exception ex) {
+      //all good
+    }
+
+  }
+
+  @Test
   public void testDuplicateContainer() throws Exception {
     try {
       // Create Container.
@@ -224,7 +309,7 @@ public class TestKeyValueContainer {
     keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
     keyValueContainer.close();
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     assertEquals(ContainerProtos.ContainerLifeCycleState.CLOSED,
@@ -249,7 +334,7 @@ public class TestKeyValueContainer {
     metadata.put("OWNER", "hdfs");
     keyValueContainer.update(metadata, true);
 
-    keyValueContainerData = (KeyValueContainerData) keyValueContainer
+    keyValueContainerData = keyValueContainer
         .getContainerData();
 
     assertEquals(2, keyValueContainerData.getMetadata().size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ca29fb75/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
new file mode 100644
index 0000000..a599f72
--- /dev/null
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
@@ -0,0 +1,231 @@
+/*
+ * 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.container.keyvalue;
+
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorInputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the tar/untar for a given container.
+ */
+public class TestTarContainerPacker {
+
+  private static final String TEST_DB_FILE_NAME = "test1";
+
+  private static final String TEST_DB_FILE_CONTENT = "test1";
+
+  private static final String TEST_CHUNK_FILE_NAME = "chunk1";
+
+  private static final String TEST_CHUNK_FILE_CONTENT = "This is a chunk";
+
+  private static final String TEST_DESCRIPTOR_FILE_CONTENT = "descriptor";
+
+  private ContainerPacker packer = new TarContainerPacker();
+
+  private static final Path SOURCE_CONTAINER_ROOT =
+      Paths.get("target/test/data/packer-source-dir");
+
+  private static final Path DEST_CONTAINER_ROOT =
+      Paths.get("target/test/data/packer-dest-dir");
+
+  @BeforeClass
+  public static void init() throws IOException {
+    initDir(SOURCE_CONTAINER_ROOT);
+    initDir(DEST_CONTAINER_ROOT);
+  }
+
+  private static void initDir(Path path) throws IOException {
+    if (path.toFile().exists()) {
+      FileUtils.deleteDirectory(path.toFile());
+    }
+    path.toFile().mkdirs();
+  }
+
+  private KeyValueContainerData createContainer(long id, Path dir,
+      OzoneConfiguration conf) throws IOException {
+
+    Path containerDir = dir.resolve("container" + id);
+    Path dbDir = containerDir.resolve("db");
+    Path dataDir = containerDir.resolve("data");
+    Files.createDirectories(dbDir);
+    Files.createDirectories(dataDir);
+
+    KeyValueContainerData containerData = new KeyValueContainerData(id, -1);
+    containerData.setChunksPath(dataDir.toString());
+    containerData.setMetadataPath(dbDir.getParent().toString());
+    containerData.setDbFile(dbDir.toFile());
+
+
+    return containerData;
+  }
+
+  @Test
+  public void pack() throws IOException, CompressorException {
+
+    //GIVEN
+    OzoneConfiguration conf = new OzoneConfiguration();
+
+    KeyValueContainerData sourceContainerData =
+        createContainer(1L, SOURCE_CONTAINER_ROOT, conf);
+
+    KeyValueContainer sourceContainer =
+        new KeyValueContainer(sourceContainerData, conf);
+
+    //sample db file in the metadata directory
+    try (FileWriter writer = new FileWriter(
+        sourceContainerData.getDbFile().toPath()
+            .resolve(TEST_DB_FILE_NAME)
+            .toFile())) {
+      IOUtils.write(TEST_DB_FILE_CONTENT, writer);
+    }
+
+    //sample chunk file in the chunk directory
+    try (FileWriter writer = new FileWriter(
+        Paths.get(sourceContainerData.getChunksPath())
+            .resolve(TEST_CHUNK_FILE_NAME)
+            .toFile())) {
+      IOUtils.write(TEST_CHUNK_FILE_CONTENT, writer);
+    }
+
+    //sample container descriptor file
+    try (FileWriter writer = new FileWriter(
+        sourceContainer.getContainerFile())) {
+      IOUtils.write(TEST_DESCRIPTOR_FILE_CONTENT, writer);
+    }
+
+    Path targetFile =
+        SOURCE_CONTAINER_ROOT.getParent().resolve("container.tar.gz");
+
+    //WHEN: pack it
+    try (FileOutputStream output = new FileOutputStream(targetFile.toFile())) {
+      packer.pack(sourceContainer, output);
+    }
+
+    //THEN: check the result
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      CompressorInputStream uncompressed = new CompressorStreamFactory()
+          .createCompressorInputStream(CompressorStreamFactory.GZIP, input);
+      TarArchiveInputStream tarStream = new TarArchiveInputStream(uncompressed);
+
+      TarArchiveEntry entry;
+      Map<String, TarArchiveEntry> entries = new HashMap<>();
+      while ((entry = tarStream.getNextTarEntry()) != null) {
+        entries.put(entry.getName(), entry);
+      }
+
+      Assert.assertTrue(
+          entries.containsKey("container.yaml"));
+
+    }
+
+    //read the container descriptor only
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      String containerYaml = new String(packer.unpackContainerDescriptor(input),
+          Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(TEST_DESCRIPTOR_FILE_CONTENT, containerYaml);
+    }
+
+    KeyValueContainerData destinationContainerData =
+        createContainer(2L, DEST_CONTAINER_ROOT, conf);
+
+    KeyValueContainer destinationContainer =
+        new KeyValueContainer(destinationContainerData, conf);
+
+    String descriptor = "";
+
+    //unpackContainerData
+    try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
+      descriptor =
+          new String(packer.unpackContainerData(destinationContainer, input),
+              Charset.forName(StandardCharsets.UTF_8.name()));
+    }
+
+    assertExampleMetadataDbIsGood(
+        destinationContainerData.getDbFile().toPath());
+    assertExampleChunkFileIsGood(
+        Paths.get(destinationContainerData.getChunksPath()));
+    Assert.assertFalse(
+        "Descriptor file should not been exctarcted by the "
+            + "unpackContainerData Call",
+        destinationContainer.getContainerFile().exists());
+    Assert.assertEquals(TEST_DESCRIPTOR_FILE_CONTENT, descriptor);
+
+  }
+
+
+  private void assertExampleMetadataDbIsGood(Path dbPath)
+      throws IOException {
+
+    Path dbFile = dbPath.resolve(TEST_DB_FILE_NAME);
+
+    Assert.assertTrue(
+        "example DB file is missing after pack/unpackContainerData: " + dbFile,
+        Files.exists(dbFile));
+
+    try (FileInputStream testFile = new FileInputStream(dbFile.toFile())) {
+      List<String> strings = IOUtils
+          .readLines(testFile, Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(1, strings.size());
+      Assert.assertEquals(TEST_DB_FILE_CONTENT, strings.get(0));
+    }
+  }
+
+  private void assertExampleChunkFileIsGood(Path chunkDirPath)
+      throws IOException {
+
+    Path chunkFile = chunkDirPath.resolve(TEST_CHUNK_FILE_NAME);
+
+    Assert.assertTrue(
+        "example chunk file is missing after pack/unpackContainerData: "
+            + chunkFile,
+        Files.exists(chunkFile));
+
+    try (FileInputStream testFile = new FileInputStream(chunkFile.toFile())) {
+      List<String> strings = IOUtils
+          .readLines(testFile, Charset.forName(StandardCharsets.UTF_8.name()));
+      Assert.assertEquals(1, strings.size());
+      Assert.assertEquals(TEST_CHUNK_FILE_CONTENT, strings.get(0));
+    }
+  }
+
+}
\ No newline at end of file


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


[44/50] [abbrv] hadoop git commit: HDFS-13848. Refactor NameNode failover proxy providers. Contributed by Konstantin Shvachko.

Posted by sh...@apache.org.
HDFS-13848. Refactor NameNode failover proxy providers. Contributed by Konstantin Shvachko.

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

Branch: refs/heads/HDFS-12943
Commit: a4121c71c29d13866a605d9c0d013e5de9c147c3
Parents: a5eba25
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Fri Aug 24 18:27:30 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Aug 24 18:27:30 2018 -0700

----------------------------------------------------------------------
 .../hadoop/io/retry/FailoverProxyProvider.java  |  15 +-
 .../ha/AbstractNNFailoverProxyProvider.java     | 152 ++++++++++++++++++-
 .../ha/ConfiguredFailoverProxyProvider.java     | 141 ++---------------
 .../namenode/ha/IPFailoverProxyProvider.java    |  51 +------
 4 files changed, 176 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
index c73e083..f2fa3af 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
@@ -30,27 +30,30 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
 @InterfaceStability.Evolving
 public interface FailoverProxyProvider<T> extends Closeable {
-  public static final class ProxyInfo<T> {
-    public final T proxy;
+  static class ProxyInfo<T> {
+    public T proxy;
     /*
      * The information (e.g., the IP address) of the current proxy object. It
      * provides information for debugging purposes.
      */
-    public final String proxyInfo;
+    public String proxyInfo;
 
     public ProxyInfo(T proxy, String proxyInfo) {
       this.proxy = proxy;
       this.proxyInfo = proxyInfo;
     }
 
+    private String proxyName() {
+      return proxy != null ? proxy.getClass().getSimpleName() : "UnknownProxy";
+    }
+
     public String getString(String methodName) {
-      return proxy.getClass().getSimpleName() + "." + methodName
-          + " over " + proxyInfo;
+      return proxyName() + "." + methodName + " over " + proxyInfo;
     }
 
     @Override
     public String toString() {
-      return proxy.getClass().getSimpleName() + " over " + proxyInfo;
+      return proxyName() + " over " + proxyInfo;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
index e0fdb32..252b70d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
@@ -18,14 +18,68 @@
 
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.HAUtilClient;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class AbstractNNFailoverProxyProvider<T> implements
     FailoverProxyProvider <T> {
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(AbstractNNFailoverProxyProvider.class);
 
-  private AtomicBoolean fallbackToSimpleAuth;
+  protected Configuration conf;
+  protected Class<T> xface;
+  protected HAProxyFactory<T> factory;
+  protected UserGroupInformation ugi;
+  protected AtomicBoolean fallbackToSimpleAuth;
+
+  protected AbstractNNFailoverProxyProvider() {
+  }
+
+  protected AbstractNNFailoverProxyProvider(Configuration conf, URI uri,
+      Class<T> xface, HAProxyFactory<T> factory) {
+    this.conf = new Configuration(conf);
+    this.xface = xface;
+    this.factory = factory;
+    try {
+      this.ugi = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    int maxRetries = this.conf.getInt(
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
+        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
+        maxRetries);
+
+    int maxRetriesOnSocketTimeouts = this.conf.getInt(
+        HdfsClientConfigKeys
+        .Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        HdfsClientConfigKeys
+        .Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic
+        .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        maxRetriesOnSocketTimeouts);
+  }
 
   /**
    * Inquire whether logical HA URI is used for the implementation. If it is
@@ -51,4 +105,100 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
   public synchronized AtomicBoolean getFallbackToSimpleAuth() {
     return fallbackToSimpleAuth;
   }
+
+  /**
+   * ProxyInfo to a NameNode. Includes its address.
+   */
+  public static class NNProxyInfo<T> extends ProxyInfo<T> {
+    private InetSocketAddress address;
+
+    public NNProxyInfo(InetSocketAddress address) {
+      super(null, address.toString());
+      this.address = address;
+    }
+
+    public InetSocketAddress getAddress() {
+      return address;
+    }
+  }
+
+  @Override
+  public Class<T> getInterface() {
+    return xface;
+  }
+
+  /**
+   * Create a proxy if it has not been created yet.
+   */
+  protected NNProxyInfo<T> createProxyIfNeeded(NNProxyInfo<T> pi) {
+    if (pi.proxy == null) {
+      assert pi.getAddress() != null : "Proxy address is null";
+      try {
+        pi.proxy = factory.createProxy(conf,
+            pi.getAddress(), xface, ugi, false, getFallbackToSimpleAuth());
+      } catch (IOException ioe) {
+        LOG.error("{} Failed to create RPC proxy to NameNode",
+            this.getClass().getSimpleName(), ioe);
+        throw new RuntimeException(ioe);
+      }
+    }
+    return pi;
+  }
+
+  /**
+   * Get list of configured NameNode proxy addresses.
+   * Randomize the list if requested.
+   */
+  protected List<NNProxyInfo<T>> getProxyAddresses(URI uri, String addressKey) {
+    final List<NNProxyInfo<T>> proxies = new ArrayList<NNProxyInfo<T>>();
+    Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtilClient.getAddresses(conf, null, addressKey);
+    Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
+
+    if (addressesInNN == null || addressesInNN.size() == 0) {
+      throw new RuntimeException("Could not find any configured addresses " +
+          "for URI " + uri);
+    }
+
+    Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
+    for (InetSocketAddress address : addressesOfNns) {
+      proxies.add(new NNProxyInfo<T>(address));
+    }
+    // Randomize the list to prevent all clients pointing to the same one
+    boolean randomized = getRandomOrder(conf, uri);
+    if (randomized) {
+      Collections.shuffle(proxies);
+    }
+
+    // The client may have a delegation token set for the logical
+    // URI of the cluster. Clone this token to apply to each of the
+    // underlying IPC addresses so that the IPC code can find it.
+    HAUtilClient.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
+    return proxies;
+  }
+
+  /**
+   * Check whether random order is configured for failover proxy provider
+   * for the namenode/nameservice.
+   *
+   * @param conf Configuration
+   * @param nameNodeUri The URI of namenode/nameservice
+   * @return random order configuration
+   */
+  public static boolean getRandomOrder(
+      Configuration conf, URI nameNodeUri) {
+    String host = nameNodeUri.getHost();
+    String configKeyWithHost = HdfsClientConfigKeys.Failover.RANDOM_ORDER
+        + "." + host;
+
+    if (conf.get(configKeyWithHost) != null) {
+      return conf.getBoolean(
+          configKeyWithHost,
+          HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
+    }
+
+    return conf.getBoolean(
+        HdfsClientConfigKeys.Failover.RANDOM_ORDER,
+        HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
index f46532a..92e75ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
@@ -19,23 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.HAUtilClient;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 
@@ -48,17 +36,9 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_RP
 public class ConfiguredFailoverProxyProvider<T> extends
     AbstractNNFailoverProxyProvider<T> {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(ConfiguredFailoverProxyProvider.class);
-
-  protected final Configuration conf;
-  protected final List<AddressRpcProxyPair<T>> proxies =
-      new ArrayList<AddressRpcProxyPair<T>>();
-  protected final UserGroupInformation ugi;
-  protected final Class<T> xface;
+  protected final List<NNProxyInfo<T>> proxies;
 
   private int currentProxyIndex = 0;
-  protected final HAProxyFactory<T> factory;
 
   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory) {
@@ -67,83 +47,8 @@ public class ConfiguredFailoverProxyProvider<T> extends
 
   public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory, String addressKey) {
-    this.xface = xface;
-    this.conf = new Configuration(conf);
-    int maxRetries = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
-        maxRetries);
-
-    int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
-    this.conf.setInt(
-            CommonConfigurationKeysPublic
-                    .IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-            maxRetriesOnSocketTimeouts);
-
-    try {
-      ugi = UserGroupInformation.getCurrentUser();
-
-      Map<String, Map<String, InetSocketAddress>> map =
-          DFSUtilClient.getAddresses(conf, null, addressKey);
-      Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
-
-      if (addressesInNN == null || addressesInNN.size() == 0) {
-        throw new RuntimeException("Could not find any configured addresses " +
-            "for URI " + uri);
-      }
-
-      Collection<InetSocketAddress> addressesOfNns = addressesInNN.values();
-      for (InetSocketAddress address : addressesOfNns) {
-        proxies.add(new AddressRpcProxyPair<T>(address));
-      }
-      // Randomize the list to prevent all clients pointing to the same one
-      boolean randomized = getRandomOrder(conf, uri);
-      if (randomized) {
-        Collections.shuffle(proxies);
-      }
-
-      // The client may have a delegation token set for the logical
-      // URI of the cluster. Clone this token to apply to each of the
-      // underlying IPC addresses so that the IPC code can find it.
-      HAUtilClient.cloneDelegationTokenForLogicalUri(ugi, uri, addressesOfNns);
-      this.factory = factory;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Check whether random order is configured for failover proxy provider
-   * for the namenode/nameservice.
-   *
-   * @param conf Configuration
-   * @param nameNodeUri The URI of namenode/nameservice
-   * @return random order configuration
-   */
-  private static boolean getRandomOrder(
-      Configuration conf, URI nameNodeUri) {
-    String host = nameNodeUri.getHost();
-    String configKeyWithHost = HdfsClientConfigKeys.Failover.RANDOM_ORDER
-        + "." + host;
-
-    if (conf.get(configKeyWithHost) != null) {
-      return conf.getBoolean(
-          configKeyWithHost,
-          HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
-    }
-
-    return conf.getBoolean(
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER,
-        HdfsClientConfigKeys.Failover.RANDOM_ORDER_DEFAULT);
-  }
-
-  @Override
-  public Class<T> getInterface() {
-    return xface;
+    super(conf, uri, xface, factory);
+    this.proxies = getProxyAddresses(uri, addressKey);
   }
 
   /**
@@ -151,21 +56,8 @@ public class ConfiguredFailoverProxyProvider<T> extends
    */
   @Override
   public synchronized ProxyInfo<T> getProxy() {
-    AddressRpcProxyPair<T> current = proxies.get(currentProxyIndex);
-    return getProxy(current);
-  }
-
-  protected ProxyInfo<T> getProxy(AddressRpcProxyPair<T> current) {
-    if (current.namenode == null) {
-      try {
-        current.namenode = factory.createProxy(conf,
-            current.address, xface, ugi, false, getFallbackToSimpleAuth());
-      } catch (IOException e) {
-        LOG.error("Failed to create RPC proxy to NameNode", e);
-        throw new RuntimeException(e);
-      }
-    }
-    return new ProxyInfo<T>(current.namenode, current.address.toString());
+    NNProxyInfo<T> current = proxies.get(currentProxyIndex);
+    return createProxyIfNeeded(current);
   }
 
   @Override
@@ -178,30 +70,17 @@ public class ConfiguredFailoverProxyProvider<T> extends
   }
 
   /**
-   * A little pair object to store the address and connected RPC proxy object to
-   * an NN. Note that {@link AddressRpcProxyPair#namenode} may be null.
-   */
-  protected static class AddressRpcProxyPair<T> {
-    public final InetSocketAddress address;
-    public T namenode;
-
-    public AddressRpcProxyPair(InetSocketAddress address) {
-      this.address = address;
-    }
-  }
-
-  /**
    * Close all the proxy objects which have been opened over the lifetime of
    * this proxy provider.
    */
   @Override
   public synchronized void close() throws IOException {
-    for (AddressRpcProxyPair<T> proxy : proxies) {
-      if (proxy.namenode != null) {
-        if (proxy.namenode instanceof Closeable) {
-          ((Closeable)proxy.namenode).close();
+    for (ProxyInfo<T> proxy : proxies) {
+      if (proxy.proxy != null) {
+        if (proxy.proxy instanceof Closeable) {
+          ((Closeable)proxy.proxy).close();
         } else {
-          RPC.stopProxy(proxy.namenode);
+          RPC.stopProxy(proxy.proxy);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4121c71/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
index ed250a0..e703740 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/IPFailoverProxyProvider.java
@@ -19,15 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSUtilClient;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * A NNFailoverProxyProvider implementation which works on IP failover setup.
@@ -47,53 +43,18 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
 public class IPFailoverProxyProvider<T> extends
     AbstractNNFailoverProxyProvider<T> {
-  private final Configuration conf;
-  private final Class<T> xface;
-  private final URI nameNodeUri;
-  private final HAProxyFactory<T> factory;
-  private ProxyInfo<T> nnProxyInfo = null;
+  private final NNProxyInfo<T> nnProxyInfo;
 
   public IPFailoverProxyProvider(Configuration conf, URI uri,
       Class<T> xface, HAProxyFactory<T> factory) {
-    this.xface = xface;
-    this.nameNodeUri = uri;
-    this.factory = factory;
-
-    this.conf = new Configuration(conf);
-    int maxRetries = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
-        maxRetries);
-
-    int maxRetriesOnSocketTimeouts = this.conf.getInt(
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        HdfsClientConfigKeys.Failover.CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
-    this.conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
-        maxRetriesOnSocketTimeouts);
-  }
-
-  @Override
-  public Class<T> getInterface() {
-    return xface;
+    super(conf, uri, xface, factory);
+    this.nnProxyInfo = new NNProxyInfo<T>(DFSUtilClient.getNNAddress(uri));
   }
 
   @Override
-  public synchronized ProxyInfo<T> getProxy() {
+  public synchronized NNProxyInfo<T> getProxy() {
     // Create a non-ha proxy if not already created.
-    if (nnProxyInfo == null) {
-      try {
-        // Create a proxy that is not wrapped in RetryProxy
-        InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
-        nnProxyInfo = new ProxyInfo<T>(factory.createProxy(conf, nnAddr, xface,
-          UserGroupInformation.getCurrentUser(), false), nnAddr.toString());
-      } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
-      }
-    }
-    return nnProxyInfo;
+    return createProxyIfNeeded(nnProxyInfo);
   }
 
   /** Nothing to do for IP failover */
@@ -106,7 +67,7 @@ public class IPFailoverProxyProvider<T> extends
    */
   @Override
   public synchronized void close() throws IOException {
-    if (nnProxyInfo == null) {
+    if (nnProxyInfo.proxy == null) {
       return;
     }
     if (nnProxyInfo.proxy instanceof Closeable) {


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


[12/50] [abbrv] hadoop git commit: HDDS-355. Disable OpenKeyDeleteService and DeleteKeysService. Contributed by Anu Engineer.

Posted by sh...@apache.org.
HDDS-355. Disable OpenKeyDeleteService and DeleteKeysService.
Contributed by Anu Engineer.


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

Branch: refs/heads/HDFS-12943
Commit: ab37423ad8debe2f050133ad97b686083531c2ea
Parents: 60ffec9
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Aug 17 11:50:46 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Aug 17 11:50:46 2018 -0700

----------------------------------------------------------------------
 .../commandhandler/TestBlockDeletion.java       |  2 ++
 .../hadoop/ozone/om/TestOzoneManager.java       | 11 +++---
 .../apache/hadoop/ozone/om/package-info.java    | 22 ++++++++++++
 .../apache/hadoop/ozone/om/KeyManagerImpl.java  | 37 ++++----------------
 4 files changed, 36 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab37423a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index badd435..45659bd 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -102,6 +103,7 @@ public class TestBlockDeletion {
   }
 
   @Test(timeout = 60000)
+  @Ignore("Until delete background service is fixed.")
   public void testBlockDeletion()
       throws IOException, InterruptedException {
     String volumeName = UUID.randomUUID().toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab37423a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
index 7c8595c..5109453 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
@@ -56,12 +56,12 @@ import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.ListKeys;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -1188,10 +1188,11 @@ public class TestOzoneManager {
   }
 
 
-  @Test
+  //Disabling this test
+  @Ignore("Disabling this test until Open Key is fixed.")
   public void testExpiredOpenKey() throws Exception {
-    BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster
-        .getOzoneManager().getKeyManager()).getOpenKeyCleanupService();
+//    BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster
+//        .getOzoneManager().getKeyManager()).getOpenKeyCleanupService();
 
     String userName = "user" + RandomStringUtils.randomNumeric(5);
     String adminName = "admin" + RandomStringUtils.randomNumeric(5);
@@ -1252,7 +1253,7 @@ public class TestOzoneManager {
     KeyArgs keyArgs5 = new KeyArgs("testKey5", bucketArgs);
     storageHandler.newKeyWriter(keyArgs5);
 
-    openKeyCleanUpService.triggerBackgroundTaskForTesting();
+    //openKeyCleanUpService.triggerBackgroundTaskForTesting();
     Thread.sleep(2000);
     // now all k1-k4 should have been removed by the clean-up task, only k5
     // should be present in ExpiredOpenKeys.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab37423a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/package-info.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/package-info.java
new file mode 100644
index 0000000..5ad6770
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Ozone Manager Tests.
+ */
+package org.apache.hadoop.ozone.om;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab37423a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 75342c6..83394f0 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.hadoop.ozone.om;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -34,7 +33,6 @@ import org.apache.hadoop.ozone.protocol.proto
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
 import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BackgroundService;
 import org.apache.hadoop.utils.BatchOperation;
 import org.iq80.leveldb.DBException;
 import org.slf4j.Logger;
@@ -91,15 +89,14 @@ public class KeyManagerImpl implements KeyManager {
   private final OMMetadataManager metadataManager;
   private final long scmBlockSize;
   private final boolean useRatis;
-  private final BackgroundService keyDeletingService;
-  private final BackgroundService openKeyCleanupService;
 
   private final long preallocateMax;
   private final Random random;
   private final String omId;
 
   public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
-                        OMMetadataManager metadataManager, OzoneConfiguration conf,
+                        OMMetadataManager metadataManager,
+                        OzoneConfiguration conf,
                         String omId) {
     this.scmBlockClient = scmBlockClient;
     this.metadataManager = metadataManager;
@@ -107,43 +104,20 @@ public class KeyManagerImpl implements KeyManager {
         OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB;
     this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
         DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
-    long  blockDeleteInterval = conf.getTimeDuration(
-        OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
-        OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT,
-        TimeUnit.MILLISECONDS);
-    long serviceTimeout = conf.getTimeDuration(
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
-        OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
-        TimeUnit.MILLISECONDS);
     this.preallocateMax = conf.getLong(
         OZONE_KEY_PREALLOCATION_MAXSIZE,
         OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT);
-    keyDeletingService = new KeyDeletingService(
-        scmBlockClient, this, blockDeleteInterval, serviceTimeout, conf);
-    int openkeyCheckInterval = conf.getInt(
-        OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS,
-        OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS_DEFAULT);
-    openKeyCleanupService = new OpenKeyCleanupService(
-        scmBlockClient, this, openkeyCheckInterval, serviceTimeout);
     random = new Random();
     this.omId = omId;
   }
 
-  @VisibleForTesting
-  public BackgroundService getOpenKeyCleanupService() {
-    return openKeyCleanupService;
-  }
 
   @Override
   public void start() {
-    keyDeletingService.start();
-    openKeyCleanupService.start();
   }
 
   @Override
   public void stop() throws IOException {
-    keyDeletingService.shutdown();
-    openKeyCleanupService.shutdown();
   }
 
   private void validateBucket(String volumeName, String bucketName)
@@ -398,7 +372,7 @@ public class KeyManagerImpl implements KeyManager {
     String bucketName = args.getBucketName();
     String fromKeyName = args.getKeyName();
     if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
-      LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.",
+      LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
           volumeName, bucketName, fromKeyName, toKeyName);
       throw new OMException("Key name is empty",
           ResultCodes.FAILED_INVALID_KEY_NAME);
@@ -446,7 +420,7 @@ public class KeyManagerImpl implements KeyManager {
       batch.put(toKey, newKeyInfo.getProtobuf().toByteArray());
       metadataManager.writeBatch(batch);
     } catch (DBException ex) {
-      LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}.",
+      LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
           volumeName, bucketName, fromKeyName, toKeyName, ex);
       throw new OMException(ex.getMessage(),
           ResultCodes.FAILED_KEY_RENAME);
@@ -487,7 +461,8 @@ public class KeyManagerImpl implements KeyManager {
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String keyPrefix, int maxKeys) throws IOException {
+                                  String startKey, String keyPrefix,
+      int maxKeys) throws IOException {
     Preconditions.checkNotNull(volumeName);
     Preconditions.checkNotNull(bucketName);
 


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


[26/50] [abbrv] hadoop git commit: YARN-7494. Add muti-node lookup mechanism and pluggable nodes sorting policies to optimize placement decision. Contributed by Sunil Govindan.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java
new file mode 100644
index 0000000..d765af8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/ResourceUsageMultiNodeLookupPolicy.java
@@ -0,0 +1,79 @@
+/**
+ * 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.server.resourcemanager.scheduler.placement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+
+import java.util.Comparator;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+/**
+ * <p>
+ * This class has the following functionality:
+ *
+ * <p>
+ * ResourceUsageMultiNodeLookupPolicy holds sorted nodes list based on the
+ * resource usage of nodes at given time.
+ * </p>
+ */
+public class ResourceUsageMultiNodeLookupPolicy<N extends SchedulerNode>
+    implements MultiNodeLookupPolicy<N> {
+
+  protected Map<String, Set<N>> nodesPerPartition = new ConcurrentHashMap<>();
+  protected Comparator<N> comparator;
+
+  public ResourceUsageMultiNodeLookupPolicy() {
+    this.comparator = new Comparator<N>() {
+      @Override
+      public int compare(N o1, N o2) {
+        int allocatedDiff = o1.getAllocatedResource()
+            .compareTo(o2.getAllocatedResource());
+        if (allocatedDiff == 0) {
+          return o1.getNodeID().compareTo(o2.getNodeID());
+        }
+        return allocatedDiff;
+      }
+    };
+  }
+
+  @Override
+  public Iterator<N> getPreferredNodeIterator(Collection<N> nodes,
+      String partition) {
+    return getNodesPerPartition(partition).iterator();
+  }
+
+  @Override
+  public void addAndRefreshNodesSet(Collection<N> nodes,
+      String partition) {
+    Set<N> nodeList = new ConcurrentSkipListSet<N>(comparator);
+    nodeList.addAll(nodes);
+    nodesPerPartition.put(partition, Collections.unmodifiableSet(nodeList));
+  }
+
+  @Override
+  public Set<N> getNodesPerPartition(String partition) {
+    return nodesPerPartition.getOrDefault(partition, Collections.emptySet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index eef86a4..09d3327 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -295,6 +296,8 @@ public class ReservationSystemTestUtil {
         });
 
     mockRmContext.setNodeLabelManager(nlm);
+    mockRmContext
+        .setMultiNodeSortingManager(mock(MultiNodeSortingManager.class));
     return mockRmContext;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
index b7b0eb7..df8309b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAppSchedulingInfo.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -118,7 +119,7 @@ public class TestAppSchedulingInfo {
     doReturn(mock(QueueMetrics.class)).when(queue).getMetrics();
     AppSchedulingInfo  info = new AppSchedulingInfo(
         appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
-        new ResourceUsage(), new HashMap<>(), null);
+        new ResourceUsage(), new HashMap<>(), mock(RMContext.class));
     Assert.assertEquals(0, info.getSchedulerKeys().size());
 
     Priority pri1 = Priority.newInstance(1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java
index 5cea3a2..60e25ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestBase.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.junit.Assert;
 
 import java.util.Set;
@@ -76,4 +77,16 @@ public class CapacitySchedulerTestBase {
         .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
         .getMemorySize() > 0);
   }
+
+  protected void waitforNMRegistered(ResourceScheduler scheduler, int nodecount,
+      int timesec) throws InterruptedException {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start < timesec * 1000) {
+      if (scheduler.getNumClusterNodes() < nodecount) {
+        Thread.sleep(100);
+      } else {
+        break;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 8d948b5..e77d8e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -106,8 +106,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyConta
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 
-import org.apache.hadoop.yarn.server.resourcemanager.placement
-    .UserGroupMappingPlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -172,7 +170,6 @@ import org.mockito.Mockito;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -4871,18 +4868,6 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
     return cs;
   }
 
-  private void waitforNMRegistered(ResourceScheduler scheduler, int nodecount,
-      int timesec) throws InterruptedException {
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start < timesec * 1000) {
-      if (scheduler.getNumClusterNodes() < nodecount) {
-        Thread.sleep(100);
-      } else {
-        break;
-      }
-    }
-  }
-
   @Test (timeout = 60000)
   public void testClearRequestsBeforeApplyTheProposal()
       throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java
new file mode 100644
index 0000000..c90af94
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java
@@ -0,0 +1,166 @@
+/**
+* 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.server.resourcemanager.scheduler.capacity;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSorter;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for Multi Node scheduling related tests.
+ */
+public class TestCapacitySchedulerMultiNodes extends CapacitySchedulerTestBase {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestCapacitySchedulerMultiNodes.class);
+  private CapacitySchedulerConfiguration conf;
+  private static final String POLICY_CLASS_NAME =
+      "org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceUsageMultiNodeLookupPolicy";
+
+  @Before
+  public void setUp() {
+    CapacitySchedulerConfiguration config =
+        new CapacitySchedulerConfiguration();
+    config.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+        DominantResourceCalculator.class.getName());
+    conf = new CapacitySchedulerConfiguration(config);
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    conf.set(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICIES,
+        "resource-based");
+    conf.set(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME,
+        "resource-based");
+    String policyName =
+        CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME
+            + ".resource-based" + ".class";
+    conf.set(policyName, POLICY_CLASS_NAME);
+    conf.setBoolean(CapacitySchedulerConfiguration.MULTI_NODE_PLACEMENT_ENABLED,
+        true);
+    conf.setInt("yarn.scheduler.minimum-allocation-mb", 512);
+    conf.setInt("yarn.scheduler.minimum-allocation-vcores", 1);
+  }
+
+  @Test
+  public void testMultiNodeSorterForScheduling() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    rm.registerNode("127.0.0.1:1234", 10 * GB);
+    rm.registerNode("127.0.0.1:1235", 10 * GB);
+    rm.registerNode("127.0.0.1:1236", 10 * GB);
+    rm.registerNode("127.0.0.1:1237", 10 * GB);
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    waitforNMRegistered(scheduler, 4, 5);
+    MultiNodeSortingManager<SchedulerNode> mns = rm.getRMContext()
+        .getMultiNodeSortingManager();
+    MultiNodeSorter<SchedulerNode> sorter = mns
+        .getMultiNodePolicy(POLICY_CLASS_NAME);
+    sorter.reSortClusterNodes();
+    Set<SchedulerNode> nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    Assert.assertEquals(4, nodes.size());
+    rm.stop();
+  }
+
+  @Test
+  public void testMultiNodeSorterForSchedulingWithOrdering() throws Exception {
+    MockRM rm = new MockRM(conf);
+    rm.start();
+    MockNM nm1 = rm.registerNode("127.0.0.1:1234", 10 * GB, 10);
+    MockNM nm2 = rm.registerNode("127.0.0.2:1235", 10 * GB, 10);
+    MockNM nm3 = rm.registerNode("127.0.0.3:1236", 10 * GB, 10);
+    MockNM nm4 = rm.registerNode("127.0.0.4:1237", 10 * GB, 10);
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    waitforNMRegistered(scheduler, 4, 5);
+
+    MultiNodeSortingManager<SchedulerNode> mns = rm.getRMContext()
+        .getMultiNodeSortingManager();
+    MultiNodeSorter<SchedulerNode> sorter = mns
+        .getMultiNodePolicy(POLICY_CLASS_NAME);
+    sorter.reSortClusterNodes();
+
+    Set<SchedulerNode> nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    Assert.assertEquals(4, nodes.size());
+
+    RMApp app1 = rm.submitApp(2048, "app-1", "user1", null, "default");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+    SchedulerNodeReport reportNm1 =
+        rm.getResourceScheduler().getNodeReport(nm1.getNodeId());
+
+    // check node report
+    Assert.assertEquals(2 * GB, reportNm1.getUsedResource().getMemorySize());
+    Assert.assertEquals(8 * GB,
+        reportNm1.getAvailableResource().getMemorySize());
+
+    // Ideally thread will invoke this, but thread operates every 1sec.
+    // Hence forcefully recompute nodes.
+    sorter.reSortClusterNodes();
+
+    RMApp app2 = rm.submitApp(1024, "app-2", "user2", null, "default");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    SchedulerNodeReport reportNm2 =
+        rm.getResourceScheduler().getNodeReport(nm2.getNodeId());
+
+    // check node report
+    Assert.assertEquals(1 * GB, reportNm2.getUsedResource().getMemorySize());
+    Assert.assertEquals(9 * GB,
+        reportNm2.getAvailableResource().getMemorySize());
+
+    // Ideally thread will invoke this, but thread operates every 1sec.
+    // Hence forcefully recompute nodes.
+    sorter.reSortClusterNodes();
+
+    // Node1 and Node2 are now having used resources. Hence ensure these 2 comes
+    // latter in the list.
+    nodes = sorter.getMultiNodeLookupPolicy()
+        .getNodesPerPartition("");
+    List<NodeId> currentNodes = new ArrayList<>();
+    currentNodes.add(nm3.getNodeId());
+    currentNodes.add(nm4.getNodeId());
+    currentNodes.add(nm2.getNodeId());
+    currentNodes.add(nm1.getNodeId());
+    Iterator<SchedulerNode> it = nodes.iterator();
+    SchedulerNode current;
+    int i = 0;
+    while (it.hasNext()) {
+      current = it.next();
+      Assert.assertEquals(current.getNodeID(), currentNodes.get(i++));
+    }
+    rm.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fc3ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
index b4ebd15..e239191 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
@@ -817,4 +817,74 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     }
     return memorySize;
   }
+
+  private long waitForNodeLabelSchedulerEventUpdate(MockRM rm, String partition,
+      long expectedNodeCount, long timeout) throws InterruptedException {
+    long start = System.currentTimeMillis();
+    long size = 0;
+    while (System.currentTimeMillis() - start < timeout) {
+      CapacityScheduler scheduler = (CapacityScheduler) rm
+          .getResourceScheduler();
+      size = scheduler.getNodeTracker().getNodesPerPartition(partition).size();
+      if (size == expectedNodeCount) {
+        return size;
+      }
+      Thread.sleep(100);
+    }
+    return size;
+  }
+
+  @Test
+  public void testNodeCountBasedOnNodeLabelsFromClusterNodeTracker()
+      throws Exception {
+    // set node -> label
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(
+        ImmutableSet.of("x", "y", "z"));
+
+    // set mapping:
+    // h1 -> x
+    // h2 -> y
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h1", 1234), toSet("x")));
+    mgr.addLabelsToNode(
+        ImmutableMap.of(NodeId.newInstance("h2", 1234), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    MockNM nm1 = rm.registerNode("h1:1234", 8000);
+    rm.registerNode("h2:1234", 8000);
+    rm.registerNode("h3:1234", 8000);
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    // Ensure that cluster node tracker is updated with correct set of node
+    // after Node registration.
+    Assert.assertEquals(2,
+        cs.getNodeTracker().getNodesPerPartition("x").size());
+    Assert.assertEquals(1, cs.getNodeTracker().getNodesPerPartition("").size());
+
+    rm.unRegisterNode(nm1);
+    rm.registerNode("h4:1234", 8000);
+
+    // Ensure that cluster node tracker is updated with correct set of node
+    // after new Node registration and old node label change.
+    Assert.assertEquals(1,
+        cs.getNodeTracker().getNodesPerPartition("x").size());
+    Assert.assertEquals(2, cs.getNodeTracker().getNodesPerPartition("").size());
+
+    mgr.replaceLabelsOnNode(
+        ImmutableMap.of(NodeId.newInstance("h2", 1234), toSet("")));
+
+    // Last node with label x is replaced by CLI or REST.
+    Assert.assertEquals(0,
+        waitForNodeLabelSchedulerEventUpdate(rm, "x", 0, 3000L));
+  }
 }


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


[46/50] [abbrv] hadoop git commit: Merge commit '4aacbfff605262aaf3dbd926258afcadc86c72c0' into HDFS-12943

Posted by sh...@apache.org.
Merge commit '4aacbfff605262aaf3dbd926258afcadc86c72c0' into HDFS-12943


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

Branch: refs/heads/HDFS-12943
Commit: d53c5997ac0bb08e1b108529faf428e7b7ccbb87
Parents: 066afcf 4aacbff
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:15:15 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:15:15 2018 -0700

----------------------------------------------------------------------
 README.txt                                      |   2 +-
 dev-support/bin/ozone-dist-layout-stitching     |   2 +
 .../main/resources/checkstyle/suppressions.xml  |   1 -
 .../hadoop-common/HadoopJNI.cmake               |   2 +
 .../key/kms/LoadBalancingKMSClientProvider.java |  43 +-
 .../org/apache/hadoop/fs/shell/FsCommand.java   |   2 +-
 .../java/org/apache/hadoop/fs/shell/Touch.java  |  85 ----
 .../apache/hadoop/fs/shell/TouchCommands.java   | 198 ++++++++
 .../org/apache/hadoop/util/ServletUtil.java     |   2 +-
 .../src/site/markdown/FileSystemShell.md        |  32 ++
 .../kms/TestLoadBalancingKMSClientProvider.java | 181 +++++++-
 .../org/apache/hadoop/fs/TestFsShellTouch.java  | 103 +++++
 .../fs/contract/AbstractContractRenameTest.java |   4 +-
 .../fs/contract/AbstractContractSeekTest.java   |   2 +-
 .../fs/contract/AbstractFSContractTestBase.java |   6 +-
 .../apache/hadoop/http/TestSSLHttpServer.java   |  54 ++-
 .../apache/hadoop/test/GenericTestUtils.java    |   8 +-
 .../src/test/resources/testConf.xml             |  51 +++
 .../src/main/resources/kms-default.xml          |   8 +
 hadoop-dist/pom.xml                             |  17 -
 hadoop-dist/src/main/compose/README.md          |  51 +++
 .../src/main/compose/ozone/docker-compose.yaml  |   6 +-
 .../main/compose/ozoneperf/docker-compose.yaml  |   6 +-
 .../apache/hadoop/hdds/scm/XceiverClient.java   |   5 +-
 .../hadoop/hdds/scm/XceiverClientGrpc.java      |   5 +-
 .../hadoop/hdds/scm/XceiverClientRatis.java     |  14 +-
 .../scm/client/ContainerOperationClient.java    |   5 +-
 hadoop-hdds/common/pom.xml                      |   2 +-
 .../hadoop/hdds/scm/XceiverClientSpi.java       |   6 +-
 .../container/common/helpers/ContainerInfo.java |  25 +-
 .../scm/container/common/helpers/Pipeline.java  |  38 +-
 .../container/common/helpers/PipelineID.java    |  88 ++++
 .../org/apache/hadoop/utils/RocksDBStore.java   |   2 +-
 .../main/java/org/apache/ratis/RatisHelper.java |  19 +-
 hadoop-hdds/common/src/main/proto/hdds.proto    |   8 +-
 .../container/common/impl/ContainerSet.java     |   3 +-
 .../CloseContainerCommandHandler.java           |   3 +-
 .../common/transport/server/XceiverServer.java  |   7 +-
 .../transport/server/XceiverServerGrpc.java     |   7 +-
 .../transport/server/XceiverServerSpi.java      |   6 +-
 .../server/ratis/ContainerStateMachine.java     | 323 ++++++++++---
 .../server/ratis/XceiverServerRatis.java        |  36 +-
 .../container/ozoneimpl/OzoneContainer.java     |   8 +-
 .../commands/CloseContainerCommand.java         |  19 +-
 .../StorageContainerDatanodeProtocol.proto      |   1 +
 .../hadoop/hdds/scm/block/BlockManagerImpl.java |   4 +-
 .../hadoop/hdds/scm/block/DeletedBlockLog.java  |   6 +-
 .../hdds/scm/block/DeletedBlockLogImpl.java     |  38 +-
 .../hdds/scm/block/PendingDeleteHandler.java    |  38 ++
 .../hdds/scm/block/PendingDeleteStatusList.java |  79 ++++
 .../hdds/scm/block/SCMBlockDeletingService.java |  20 +-
 .../container/CloseContainerEventHandler.java   |   2 +-
 .../hdds/scm/container/ContainerMapping.java    |  75 ++-
 .../scm/container/ContainerStateManager.java    |  28 +-
 .../scm/container/closer/ContainerCloser.java   |   4 +-
 .../scm/container/states/ContainerStateMap.java |  18 +-
 .../hadoop/hdds/scm/events/SCMEvents.java       |   9 +
 .../hdds/scm/exceptions/SCMException.java       |   3 +-
 .../hadoop/hdds/scm/node/DatanodeInfo.java      |   8 +
 .../hdds/scm/pipelines/PipelineManager.java     |  36 +-
 .../hdds/scm/pipelines/PipelineSelector.java    |  31 +-
 .../scm/pipelines/ratis/RatisManagerImpl.java   |  18 +-
 .../standalone/StandaloneManagerImpl.java       |  14 +-
 .../scm/server/StorageContainerManager.java     |   5 +
 .../hadoop/hdds/scm/block/TestBlockManager.java |  25 -
 .../hdds/scm/block/TestDeletedBlockLog.java     |   6 +-
 .../scm/container/TestContainerMapping.java     |  50 ++
 .../container/TestContainerReportHandler.java   |   4 +-
 .../replication/TestReplicationManager.java     |   5 +-
 .../hadoop/hdds/scm/node/TestNodeManager.java   |   4 +-
 .../ozone/container/common/TestEndPoint.java    |   2 +
 .../hadoop/hdfs/DistributedFileSystem.java      |   6 +-
 .../hadoop-hdfs-native-client/pom.xml           | 105 +++--
 .../src/CMakeLists.txt                          |  48 ++
 .../src/main/native/libhdfs/CMakeLists.txt      |   6 +-
 .../main/native/libhdfspp/tests/CMakeLists.txt  |   4 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |   4 +
 .../server/blockmanagement/BlockManager.java    |   4 +
 .../server/blockmanagement/DatanodeManager.java |   7 +-
 .../hdfs/server/datanode/BPServiceActor.java    |   3 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  15 +
 .../hdfs/server/datanode/DataNodeMXBean.java    |   7 +
 .../hdfs/server/datanode/DirectoryScanner.java  |   2 +-
 .../server/namenode/FSImageFormatPBINode.java   |  28 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  10 +-
 .../server/namenode/FSPermissionChecker.java    |   2 +-
 .../hdfs/server/namenode/SecondaryNameNode.java |   5 +
 .../namenode/SecondaryNameNodeInfoMXBean.java   |   7 +
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |   6 +
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   |  12 +-
 .../src/main/native/docs/libhdfs_footer.html    |   2 +-
 .../src/main/webapps/hdfs/explorer.js           |   2 +-
 .../src/site/markdown/HDFSErasureCoding.md      |   8 +-
 .../hadoop/hdfs/TestDFSClientFailover.java      |   6 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |   7 +
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |   6 +-
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |  65 +++
 .../org/apache/hadoop/hdfs/TestReplication.java |   6 +-
 .../qjournal/client/TestIPCLoggerChannel.java   |   6 +-
 .../server/datanode/BlockReportTestBase.java    |   8 +-
 .../hdfs/server/datanode/DataNodeTestUtils.java |   3 +
 .../server/datanode/TestDataNodeMXBean.java     |  47 +-
 .../server/datanode/TestDirectoryScanner.java   |  42 +-
 .../hdfs/server/namenode/FSImageTestUtil.java   |  13 +
 .../hdfs/server/namenode/TestCheckpoint.java    |   4 +-
 .../hdfs/server/namenode/TestDeleteRace.java    |  10 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  18 +
 .../namenode/TestINodeAttributeProvider.java    |  43 +-
 .../TestNameNodeMetadataConsistency.java        |   2 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  10 +-
 .../server/namenode/TestSecureNameNode.java     |  52 ++-
 .../hdfs/server/namenode/ha/TestDNFencing.java  |   6 +-
 .../namenode/ha/TestPipelinesFailover.java      |  10 +-
 .../namenode/ha/TestStandbyCheckpoints.java     |   6 +-
 .../security/TestRefreshUserMappings.java       |  51 ++-
 .../test/resources/testErasureCodingConf.xml    |   6 +-
 hadoop-mapreduce-project/pom.xml                |   2 +-
 hadoop-ozone/docs/content/GettingStarted.md     |   2 +-
 hadoop-ozone/docs/pom.xml                       |  17 +
 hadoop-ozone/docs/static/OzoneOverview.svg      |  13 +
 .../hdds/scm/pipeline/TestNode2PipelineMap.java |   8 +-
 .../hdds/scm/pipeline/TestPipelineClose.java    |  10 +-
 .../hadoop/ozone/TestMiniOzoneCluster.java      |   3 +-
 .../ozone/container/ContainerTestHelper.java    |   5 +-
 .../commandhandler/TestBlockDeletion.java       |  57 ++-
 .../TestCloseContainerByPipeline.java           |  41 +-
 .../TestCloseContainerHandler.java              |   7 +-
 .../server/TestContainerStateMachine.java       | 201 ++++++++
 .../hadoop/ozone/om/TestOzoneManager.java       |  11 +-
 .../apache/hadoop/ozone/om/package-info.java    |  22 +
 .../apache/hadoop/ozone/om/KeyManagerImpl.java  |  37 +-
 .../web/ozShell/volume/UpdateVolumeHandler.java |   5 +
 .../ITestOzoneContractGetFileStatus.java        |   2 +-
 hadoop-ozone/pom.xml                            |  10 -
 .../genesis/BenchMarkContainerStateMap.java     |  13 +-
 .../genesis/BenchMarkDatanodeDispatcher.java    |   3 +-
 hadoop-project/pom.xml                          |   4 +-
 .../fs/aliyun/oss/AliyunOSSInputStream.java     |   6 +-
 .../apache/hadoop/tools/HadoopArchiveLogs.java  |   6 +-
 .../hadoop/tools/HadoopArchiveLogsRunner.java   |   8 +-
 .../org/apache/hadoop/tools/HadoopArchives.java |   6 +-
 .../hadoop/fs/s3a/s3guard/DDBPathMetadata.java  |  77 ++++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   | 130 ++++--
 .../PathMetadataDynamoDBTranslation.java        |  71 ++-
 .../apache/hadoop/fs/s3a/s3guard/S3Guard.java   |   4 +
 .../site/markdown/tools/hadoop-aws/s3guard.md   |   5 +-
 .../site/markdown/tools/hadoop-aws/testing.md   | 157 +++++++
 .../s3a/ITestS3AContractGetFileStatus.java      |   2 +-
 .../ITestS3AContractGetFileStatusV1List.java    |   2 +-
 .../s3guard/AbstractS3GuardToolTestBase.java    |  13 +-
 .../s3a/s3guard/ITestS3GuardToolDynamoDB.java   |   7 +-
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   |  11 +
 .../fs/s3a/s3guard/MetadataStoreTestBase.java   |  49 ++
 .../TestPathMetadataDynamoDBTranslation.java    |  47 +-
 .../hadoop/contrib/utils/join/JobBase.java      |   6 +-
 .../org/apache/hadoop/tools/CopyListing.java    |   6 +-
 .../java/org/apache/hadoop/tools/DistCp.java    |   6 +-
 .../apache/hadoop/tools/GlobbedCopyListing.java |   6 +-
 .../org/apache/hadoop/tools/OptionsParser.java  |   6 +-
 .../apache/hadoop/tools/RegexCopyFilter.java    |   8 +-
 .../apache/hadoop/tools/SimpleCopyListing.java  |  10 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  |   6 +-
 .../tools/mapred/RetriableFileCopyCommand.java  |   8 +-
 .../tools/mapred/UniformSizeInputFormat.java    |   8 +-
 .../tools/mapred/lib/DynamicInputChunk.java     |   8 +-
 .../mapred/lib/DynamicInputChunkContext.java    |   6 +-
 .../tools/mapred/lib/DynamicInputFormat.java    |   6 +-
 .../tools/mapred/lib/DynamicRecordReader.java   |   6 +-
 .../apache/hadoop/tools/util/DistCpUtils.java   |   6 +-
 .../hadoop/tools/util/ProducerConsumer.java     |   6 +-
 .../hadoop/tools/util/RetriableCommand.java     |   6 +-
 .../apache/hadoop/tools/TestCopyListing.java    |   6 +-
 .../apache/hadoop/tools/TestDistCpSystem.java   |   8 +-
 .../apache/hadoop/tools/TestDistCpViewFs.java   |   6 +-
 .../apache/hadoop/tools/TestExternalCall.java   |   6 +-
 .../hadoop/tools/TestFileBasedCopyListing.java  |   6 +-
 .../apache/hadoop/tools/TestIntegration.java    |   6 +-
 .../contract/AbstractContractDistCpTest.java    |   2 +-
 .../hadoop/tools/mapred/TestCopyCommitter.java  |   6 +-
 .../hadoop/tools/mapred/TestCopyMapper.java     |   6 +-
 .../tools/mapred/TestCopyOutputFormat.java      |   6 +-
 .../mapred/lib/TestDynamicInputFormat.java      |   6 +-
 .../hadoop/tools/util/TestDistCpUtils.java      |   6 +-
 .../tools/util/TestThrottledInputStream.java    |   8 +-
 .../java/org/apache/hadoop/tools/DistTool.java  |   6 +-
 .../mapred/gridmix/ClusterSummarizer.java       |   6 +-
 .../gridmix/CompressionEmulationUtil.java       |   6 +-
 .../gridmix/DistributedCacheEmulator.java       |   8 +-
 .../hadoop/mapred/gridmix/EchoUserResolver.java |   6 +-
 .../mapred/gridmix/ExecutionSummarizer.java     |   6 +-
 .../apache/hadoop/mapred/gridmix/FilePool.java  |   6 +-
 .../hadoop/mapred/gridmix/GridmixJob.java       |   6 +-
 .../hadoop/mapred/gridmix/InputStriper.java     |   6 +-
 .../hadoop/mapred/gridmix/JobFactory.java       |   6 +-
 .../hadoop/mapred/gridmix/JobMonitor.java       |   6 +-
 .../hadoop/mapred/gridmix/JobSubmitter.java     |   6 +-
 .../apache/hadoop/mapred/gridmix/LoadJob.java   |   6 +-
 .../mapred/gridmix/RandomTextDataGenerator.java |   6 +-
 .../hadoop/mapred/gridmix/ReplayJobFactory.java |   6 +-
 .../mapred/gridmix/RoundRobinUserResolver.java  |   6 +-
 .../hadoop/mapred/gridmix/SerialJobFactory.java |   6 +-
 .../apache/hadoop/mapred/gridmix/SleepJob.java  |   6 +-
 .../hadoop/mapred/gridmix/Statistics.java       |   6 +-
 .../hadoop/mapred/gridmix/StressJobFactory.java |   6 +-
 .../mapred/gridmix/SubmitterUserResolver.java   |   6 +-
 .../hadoop/mapred/gridmix/CommonJobTest.java    |   6 +-
 .../hadoop/mapred/gridmix/DebugJobProducer.java |   6 +-
 .../hadoop/mapred/gridmix/GridmixTestUtils.java |   6 +-
 .../hadoop/mapred/gridmix/TestFilePool.java     |   6 +-
 .../hadoop/mapred/gridmix/TestFileQueue.java    |   6 +-
 .../mapred/gridmix/TestGridMixClasses.java      |   6 +-
 .../mapred/gridmix/TestGridmixRecord.java       |   6 +-
 .../mapred/gridmix/TestRecordFactory.java       |   6 +-
 .../tools/rumen/DeskewedJobTraceReader.java     |   8 +-
 .../org/apache/hadoop/tools/rumen/Folder.java   |   6 +-
 .../hadoop/tools/rumen/HadoopLogsAnalyzer.java  |   6 +-
 .../hadoop/tools/rumen/HistoryEventEmitter.java |   6 +-
 .../apache/hadoop/tools/rumen/JobBuilder.java   |   6 +-
 .../apache/hadoop/tools/rumen/ParsedJob.java    |   6 +-
 .../apache/hadoop/tools/rumen/ParsedTask.java   |   6 +-
 .../hadoop/tools/rumen/ParsedTaskAttempt.java   |   6 +-
 .../hadoop/tools/rumen/RandomSeedGenerator.java |   6 +-
 .../apache/hadoop/tools/rumen/TraceBuilder.java |   8 +-
 .../apache/hadoop/tools/rumen/ZombieJob.java    |   6 +-
 .../sls/resourcemanager/MockAMLauncher.java     |   6 +-
 .../hadoop/yarn/sls/synthetic/SynthJob.java     |   6 +-
 .../sls/synthetic/SynthTraceJobProducer.java    |   6 +-
 .../org/apache/hadoop/streaming/PipeMapRed.java |  17 +-
 .../streaming/StreamBaseRecordReader.java       |   5 +-
 .../org/apache/hadoop/streaming/StreamJob.java  |   6 +-
 .../mapreduce/StreamBaseRecordReader.java       |   8 +-
 .../streaming/TestMultipleArchiveFiles.java     |   6 +-
 .../streaming/TestStreamXmlMultipleRecords.java |   6 +-
 .../streaming/TestStreamingBadRecords.java      |   8 +-
 .../org/apache/hadoop/streaming/UtilTest.java   |   6 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +-
 .../hadoop-yarn-services-api/pom.xml            |  57 +++
 .../yarn/service/client/ApiServiceClient.java   |  83 +++-
 .../client/TestSecureApiServiceClient.java      |  83 ++++
 .../yarn/service/component/Component.java       |   3 +-
 .../component/instance/ComponentInstance.java   |   2 +-
 .../containerlaunch/AbstractLauncher.java       |   5 +
 .../containerlaunch/ContainerLaunchService.java |  11 +
 .../provider/AbstractProviderService.java       |   5 +-
 .../defaultImpl/DefaultProviderService.java     |   4 +-
 .../provider/docker/DockerProviderService.java  |  30 +-
 .../tarball/TarballProviderService.java         |   7 +-
 .../instance/TestComponentInstance.java         |   4 +-
 .../provider/TestAbstractProviderService.java   | 138 ++++++
 .../hadoop-yarn-submarine/README.md             |  53 +++
 .../hadoop-yarn-submarine/pom.xml               | 213 +++++++++
 .../yarn/submarine/client/cli/AbstractCli.java  |  47 ++
 .../hadoop/yarn/submarine/client/cli/Cli.java   | 106 +++++
 .../yarn/submarine/client/cli/CliConstants.java |  48 ++
 .../yarn/submarine/client/cli/CliUtils.java     | 174 +++++++
 .../yarn/submarine/client/cli/RunJobCli.java    | 204 +++++++++
 .../yarn/submarine/client/cli/ShowJobCli.java   | 125 +++++
 .../client/cli/param/BaseParameters.java        |  56 +++
 .../client/cli/param/RunJobParameters.java      | 222 +++++++++
 .../client/cli/param/RunParameters.java         | 103 +++++
 .../client/cli/param/ShowJobParameters.java     |  18 +
 .../yarn/submarine/common/ClientContext.java    |  77 ++++
 .../hadoop/yarn/submarine/common/Envs.java      |  27 ++
 .../common/api/JobComponentStatus.java          |  73 +++
 .../yarn/submarine/common/api/JobState.java     |  52 +++
 .../yarn/submarine/common/api/JobStatus.java    |  87 ++++
 .../yarn/submarine/common/api/TaskType.java     |  32 ++
 .../api/builder/JobComponentStatusBuilder.java  |  44 ++
 .../common/api/builder/JobStatusBuilder.java    |  64 +++
 .../common/conf/SubmarineConfiguration.java     |  51 +++
 .../submarine/common/conf/SubmarineLogs.java    |  31 ++
 .../common/exception/SubmarineException.java    |  21 +
 .../exception/SubmarineRuntimeException.java    |  25 +
 .../fs/DefaultRemoteDirectoryManager.java       |  84 ++++
 .../common/fs/RemoteDirectoryManager.java       |  30 ++
 .../yarn/submarine/runtimes/RuntimeFactory.java | 106 +++++
 .../common/FSBasedSubmarineStorageImpl.java     | 106 +++++
 .../submarine/runtimes/common/JobMonitor.java   |  84 ++++
 .../submarine/runtimes/common/JobSubmitter.java |  36 ++
 .../runtimes/common/StorageKeyConstants.java    |  24 +
 .../runtimes/common/SubmarineStorage.java       |  57 +++
 .../yarnservice/YarnServiceJobMonitor.java      |  46 ++
 .../yarnservice/YarnServiceJobSubmitter.java    | 458 +++++++++++++++++++
 .../yarnservice/YarnServiceRuntimeFactory.java  |  44 ++
 .../runtimes/yarnservice/YarnServiceUtils.java  |  78 ++++
 .../src/site/DeveloperGuide.md                  |  26 ++
 .../src/site/QuickStart.md                      | 134 ++++++
 .../client/cli/TestRunJobCliParsing.java        | 229 ++++++++++
 .../client/cli/TestShowJobCliParsing.java       | 104 +++++
 .../yarnservice/TestYarnServiceRunJobCli.java   | 167 +++++++
 .../yarnservice/YarnServiceCliTestUtils.java    |  35 ++
 .../submarine/common/MockClientContext.java     |  56 +++
 .../common/fs/MockRemoteDirectoryManager.java   |  83 ++++
 .../runtimes/common/MemorySubmarineStorage.java |  74 +++
 .../common/TestFSBasedSubmarineStorage.java     |  73 +++
 .../yarnservice/TestTFConfigGenerator.java      |  42 ++
 .../src/test/resources/core-site.xml            |  21 +
 .../src/test/resources/hdfs-site.xml            |  21 +
 .../hadoop-yarn-applications/pom.xml            |   1 +
 .../logaggregation/AggregatedLogFormat.java     |   3 +-
 .../filecontroller/LogAggregationHtmlBlock.java |  61 +++
 .../ifile/IndexedFileAggregatedLogsBlock.java   | 136 ++----
 .../tfile/TFileAggregatedLogsBlock.java         |  56 +--
 .../server/nodemanager/ContainerExecutor.java   |  62 ++-
 .../yarn/server/nodemanager/NodeManager.java    |  20 +-
 .../server/nodemanager/NodeManagerMXBean.java   |  38 ++
 .../launcher/ContainerLaunch.java               |  51 ++-
 .../runtime/DockerLinuxContainerRuntime.java    |   4 +-
 .../impl/container-executor.c                   |  92 ++--
 .../impl/container-executor.h                   |   3 +-
 .../main/native/container-executor/impl/main.c  |  26 +-
 .../nodemanager/TestContainerExecutor.java      |  31 +-
 .../nodemanager/TestNodeManagerMXBean.java      |  56 +++
 .../runtime/TestDockerContainerRuntime.java     |   8 +-
 .../server/resourcemanager/ResourceManager.java |  18 +-
 .../resourcemanager/ResourceManagerMXBean.java  |  38 ++
 .../TestResourceManagerMXBean.java              |  56 +++
 .../TestAMLaunchFailure.java                    |   2 +-
 .../TestSchedulerNegotiator.java                |   2 +-
 .../TestTimelineServiceClientIntegration.java   |   3 +-
 .../security/TestTimelineAuthFilterForV2.java   |   2 +-
 .../PerNodeTimelineCollectorsAuxService.java    |  13 +-
 323 files changed, 8429 insertions(+), 1311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d53c5997/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d53c5997/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------


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


[41/50] [abbrv] hadoop git commit: HDDS-363. Faster datanode registration during the first startup. Contributed by Elek, Marton.

Posted by sh...@apache.org.
HDDS-363. Faster datanode registration during the first startup. Contributed by Elek, Marton.


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

Branch: refs/heads/HDFS-12943
Commit: 138b0c1443ae485ba90b953362d6dea89c77a758
Parents: 55b6931
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 15:36:10 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 15:36:10 2018 +0200

----------------------------------------------------------------------
 .../statemachine/DatanodeStateMachine.java      |  5 +---
 .../common/statemachine/StateContext.java       | 20 ++++++++++++++
 .../states/datanode/RunningDatanodeState.java   |  1 +
 .../states/endpoint/RegisterEndpointTask.java   | 28 ++++++++++++++++----
 .../ozone/container/common/TestEndPoint.java    |  3 ++-
 5 files changed, 47 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index cf6f1ca..0a23912 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -48,8 +48,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
-
 /**
  * State Machine Class.
  */
@@ -60,7 +58,6 @@ public class DatanodeStateMachine implements Closeable {
   private final ExecutorService executorService;
   private final Configuration conf;
   private final SCMConnectionManager connectionManager;
-  private final long heartbeatFrequency;
   private StateContext context;
   private final OzoneContainer container;
   private DatanodeDetails datanodeDetails;
@@ -86,7 +83,6 @@ public class DatanodeStateMachine implements Closeable {
             .setNameFormat("Datanode State Machine Thread - %d").build());
     connectionManager = new SCMConnectionManager(conf);
     context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
-    heartbeatFrequency = getScmHeartbeatInterval(conf);
     container = new OzoneContainer(this.datanodeDetails,
         new OzoneConfiguration(conf), context);
     nextHB = new AtomicLong(Time.monotonicNow());
@@ -147,6 +143,7 @@ public class DatanodeStateMachine implements Closeable {
     while (context.getState() != DatanodeStates.SHUTDOWN) {
       try {
         LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
+        long heartbeatFrequency = context.getHeartbeatFrequency();
         nextHB.set(Time.monotonicNow() + heartbeatFrequency);
         context.execute(executorService, heartbeatFrequency,
             TimeUnit.MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
index 19c9496..a342294 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.ozone.protocol.commands.CommandStatus;
 import org.apache.hadoop.ozone.protocol.commands.CommandStatus
     .CommandStatusBuilder;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import static org.apache.hadoop.hdds.scm.HddsServerUtil.getScmHeartbeatInterval;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,6 +69,13 @@ public class StateContext {
   private DatanodeStateMachine.DatanodeStates state;
 
   /**
+   * Starting with a 2 sec heartbeat frequency which will be updated to the
+   * real HB frequency after scm registration. With this method the
+   * initial registration could be significant faster.
+   */
+  private AtomicLong heartbeatFrequency = new AtomicLong(2000);
+
+  /**
    * Constructs a StateContext.
    *
    * @param conf   - Configration
@@ -398,4 +407,15 @@ public class StateContext {
     }
     return false;
   }
+
+  public void configureHeartbeatFrequency(){
+    heartbeatFrequency.set(getScmHeartbeatInterval(conf));
+  }
+
+  /**
+   * Return current heartbeat frequency in ms.
+   */
+  public long getHeartbeatFrequency() {
+    return heartbeatFrequency.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 1758c03..ec2358a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -101,6 +101,7 @@ public class RunningDatanodeState implements DatanodeState {
       return  RegisterEndpointTask.newBuilder()
           .setConfig(conf)
           .setEndpointStateMachine(endpoint)
+          .setContext(context)
           .setDatanodeDetails(context.getParent().getDatanodeDetails())
           .setOzoneContainer(context.getParent().getContainer())
           .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
index 25af4a1..ccab095 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/RegisterEndpointTask.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,6 +51,7 @@ public final class RegisterEndpointTask implements
   private Future<EndpointStateMachine.EndPointStates> result;
   private DatanodeDetails datanodeDetails;
   private final OzoneContainer datanodeContainerManager;
+  private StateContext stateContext;
 
   /**
    * Creates a register endpoint task.
@@ -60,10 +62,12 @@ public final class RegisterEndpointTask implements
    */
   @VisibleForTesting
   public RegisterEndpointTask(EndpointStateMachine rpcEndPoint,
-      Configuration conf, OzoneContainer ozoneContainer) {
+      Configuration conf, OzoneContainer ozoneContainer,
+      StateContext context) {
     this.rpcEndPoint = rpcEndPoint;
     this.conf = conf;
     this.datanodeContainerManager = ozoneContainer;
+    this.stateContext = context;
 
   }
 
@@ -124,6 +128,7 @@ public final class RegisterEndpointTask implements
           rpcEndPoint.getState().getNextState();
       rpcEndPoint.setState(nextState);
       rpcEndPoint.zeroMissedCount();
+      this.stateContext.configureHeartbeatFrequency();
     } catch (IOException ex) {
       rpcEndPoint.logIfNeeded(ex);
     } finally {
@@ -150,6 +155,7 @@ public final class RegisterEndpointTask implements
     private Configuration conf;
     private DatanodeDetails datanodeDetails;
     private OzoneContainer container;
+    private StateContext context;
 
     /**
      * Constructs the builder class.
@@ -200,6 +206,10 @@ public final class RegisterEndpointTask implements
       return this;
     }
 
+    public Builder setContext(StateContext stateContext) {
+      this.context = stateContext;
+      return this;
+    }
 
     public RegisterEndpointTask build() {
       if (endPointStateMachine == null) {
@@ -210,8 +220,9 @@ public final class RegisterEndpointTask implements
 
       if (conf == null) {
         LOG.error("No config specified.");
-        throw new IllegalArgumentException("A valid configration is needed to" +
-            " construct RegisterEndpoint task");
+        throw new IllegalArgumentException(
+            "A valid configuration is needed to construct RegisterEndpoint "
+                + "task");
       }
 
       if (datanodeDetails == null) {
@@ -223,13 +234,20 @@ public final class RegisterEndpointTask implements
       if (container == null) {
         LOG.error("Container is not specified");
         throw new IllegalArgumentException("Container is not specified to " +
-            "constrict RegisterEndpoint task");
+            "construct RegisterEndpoint task");
+      }
+
+      if (context == null) {
+        LOG.error("StateContext is not specified");
+        throw new IllegalArgumentException("Container is not specified to " +
+            "construct RegisterEndpoint task");
       }
 
       RegisterEndpointTask task = new RegisterEndpointTask(this
-          .endPointStateMachine, this.conf, this.container);
+          .endPointStateMachine, this.conf, this.container, this.context);
       task.setDatanodeDetails(datanodeDetails);
       return task;
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/138b0c14/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 5071d8d..5efcdd1 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -309,7 +309,8 @@ public class TestEndPoint {
     when(ozoneContainer.getContainerReport()).thenReturn(
         TestUtils.getRandomContainerReports(10));
     RegisterEndpointTask endpointTask =
-        new RegisterEndpointTask(rpcEndPoint, conf, ozoneContainer);
+        new RegisterEndpointTask(rpcEndPoint, conf, ozoneContainer,
+            mock(StateContext.class));
     if (!clearDatanodeDetails) {
       DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
       endpointTask.setDatanodeDetails(datanodeDetails);


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


[08/50] [abbrv] hadoop git commit: HADOOP-15674. Test failure TestSSLHttpServer.testExcludedCiphers with TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 cipher suite. Contributed by Szilard Nemeth.

Posted by sh...@apache.org.
HADOOP-15674. Test failure TestSSLHttpServer.testExcludedCiphers with TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 cipher suite. Contributed by Szilard Nemeth.


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

Branch: refs/heads/HDFS-12943
Commit: 8d7c93186e3090b19aa59006bb6b32ba929bd8e6
Parents: fb5b3dc
Author: Xiao Chen <xi...@apache.org>
Authored: Fri Aug 17 10:08:52 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Fri Aug 17 10:09:23 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/http/TestSSLHttpServer.java   | 54 +++++++++++++++++++-
 1 file changed, 53 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d7c9318/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
index 5af6d6f..2166464 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
@@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory;
 
 /**
  * This testcase issues SSL certificates configures the HttpServer to serve
- * HTTPS using the created certficates and calls an echo servlet using the
+ * HTTPS using the created certificates and calls an echo servlet using the
  * corresponding HTTPS URL.
  */
 public class TestSSLHttpServer extends HttpServerFunctionalTest {
@@ -58,11 +58,15 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(TestSSLHttpServer.class);
+  private static final String HTTPS_CIPHER_SUITES_KEY = "https.cipherSuites";
+  private static final String JAVAX_NET_DEBUG_KEY = "javax.net.debug";
   private static Configuration conf;
   private static HttpServer2 server;
   private static String keystoresDir;
   private static String sslConfDir;
   private static SSLFactory clientSslFactory;
+  private static String cipherSuitesPropertyValue;
+  private static String sslDebugPropertyValue;
   private static final String excludeCiphers = "TLS_ECDHE_RSA_WITH_RC4_128_SHA,"
       + "SSL_DHE_RSA_EXPORT_WITH_DES40_CBC_SHA, \n"
       + "SSL_RSA_WITH_DES_CBC_SHA,"
@@ -83,6 +87,9 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
 
   @BeforeClass
   public static void setup() throws Exception {
+    turnOnSSLDebugLogging();
+    storeHttpsCipherSuites();
+
     conf = new Configuration();
     conf.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, 10);
 
@@ -127,6 +134,51 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
     FileUtil.fullyDelete(new File(BASEDIR));
     KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
     clientSslFactory.destroy();
+    restoreHttpsCipherSuites();
+    restoreSSLDebugLogging();
+  }
+
+  /**
+   * Stores the JVM property value of https.cipherSuites and sets its
+   * value to an empty string.
+   * This ensures that the value https.cipherSuites does
+   * not affect the result of tests.
+   */
+  private static void storeHttpsCipherSuites() {
+    String cipherSuites = System.getProperty(HTTPS_CIPHER_SUITES_KEY);
+    if (cipherSuites != null) {
+      LOG.info(
+          "Found value for property {}: {}", HTTPS_CIPHER_SUITES_KEY,
+          cipherSuites);
+      cipherSuitesPropertyValue = cipherSuites;
+    }
+    System.clearProperty(HTTPS_CIPHER_SUITES_KEY);
+  }
+
+  private static void restoreHttpsCipherSuites() {
+    if (cipherSuitesPropertyValue != null) {
+      LOG.info("Restoring property {} to value: {}", HTTPS_CIPHER_SUITES_KEY,
+          cipherSuitesPropertyValue);
+      System.setProperty(HTTPS_CIPHER_SUITES_KEY, cipherSuitesPropertyValue);
+      cipherSuitesPropertyValue = null;
+    }
+  }
+
+  private static void turnOnSSLDebugLogging() {
+    String sslDebug = System.getProperty(JAVAX_NET_DEBUG_KEY);
+    if (sslDebug != null) {
+      sslDebugPropertyValue = sslDebug;
+    }
+    System.setProperty(JAVAX_NET_DEBUG_KEY, "all");
+  }
+
+  private static void restoreSSLDebugLogging() {
+    if (sslDebugPropertyValue != null) {
+      System.setProperty(JAVAX_NET_DEBUG_KEY, sslDebugPropertyValue);
+      sslDebugPropertyValue = null;
+    } else {
+      System.clearProperty(JAVAX_NET_DEBUG_KEY);
+    }
   }
 
   @Test


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


[09/50] [abbrv] hadoop git commit: HADOOP-14154 Persist isAuthoritative bit in DynamoDBMetaStore (Contributed by Gabor Bota)

Posted by sh...@apache.org.
HADOOP-14154 Persist isAuthoritative bit in DynamoDBMetaStore (Contributed by Gabor Bota)


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

Branch: refs/heads/HDFS-12943
Commit: d7232857d8d1e10cdac171acdc931187e45fd6be
Parents: 8d7c931
Author: Aaron Fabbri <fa...@apache.org>
Authored: Fri Aug 17 10:08:30 2018 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Fri Aug 17 10:15:39 2018 -0700

----------------------------------------------------------------------
 .../hadoop/fs/s3a/s3guard/DDBPathMetadata.java  |  77 +++++++++++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   | 130 +++++++++++++++----
 .../PathMetadataDynamoDBTranslation.java        |  71 ++++++++--
 .../apache/hadoop/fs/s3a/s3guard/S3Guard.java   |   4 +
 .../site/markdown/tools/hadoop-aws/s3guard.md   |   5 +-
 .../fs/s3a/s3guard/MetadataStoreTestBase.java   |  49 +++++++
 .../TestPathMetadataDynamoDBTranslation.java    |  47 ++++++-
 7 files changed, 337 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java
new file mode 100644
index 0000000..a67fc4e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DDBPathMetadata.java
@@ -0,0 +1,77 @@
+/*
+ * 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.fs.s3a.s3guard;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.s3a.Tristate;
+
+/**
+ * {@code DDBPathMetadata} wraps {@link PathMetadata} and adds the
+ * isAuthoritativeDir flag to provide support for authoritative directory
+ * listings in {@link DynamoDBMetadataStore}.
+ */
+public class DDBPathMetadata extends PathMetadata {
+
+  private boolean isAuthoritativeDir;
+
+  public DDBPathMetadata(PathMetadata pmd, boolean isAuthoritativeDir) {
+    super(pmd.getFileStatus(), pmd.isEmptyDirectory(), pmd.isDeleted());
+    this.isAuthoritativeDir = isAuthoritativeDir;
+  }
+
+  public DDBPathMetadata(PathMetadata pmd) {
+    super(pmd.getFileStatus(), pmd.isEmptyDirectory(), pmd.isDeleted());
+    this.isAuthoritativeDir = false;
+  }
+
+  public DDBPathMetadata(FileStatus fileStatus) {
+    super(fileStatus);
+    this.isAuthoritativeDir = false;
+  }
+
+  public DDBPathMetadata(FileStatus fileStatus, Tristate isEmptyDir,
+      boolean isDeleted) {
+    super(fileStatus, isEmptyDir, isDeleted);
+    this.isAuthoritativeDir = false;
+  }
+
+  public DDBPathMetadata(FileStatus fileStatus, Tristate isEmptyDir,
+      boolean isDeleted, boolean isAuthoritativeDir) {
+    super(fileStatus, isEmptyDir, isDeleted);
+    this.isAuthoritativeDir = isAuthoritativeDir;
+  }
+
+  public boolean isAuthoritativeDir() {
+    return isAuthoritativeDir;
+  }
+
+  public void setAuthoritativeDir(boolean authoritativeDir) {
+    isAuthoritativeDir = authoritativeDir;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index ba80b88..ddb493f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -28,11 +28,16 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.auth.AWSCredentialsProvider;
@@ -422,7 +427,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
     boolean idempotent = S3AFileSystem.DELETE_CONSIDERED_IDEMPOTENT;
     if (tombstone) {
       Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem(
-          PathMetadata.tombstone(path));
+          new DDBPathMetadata(PathMetadata.tombstone(path)));
       invoker.retry("Put tombstone", path.toString(), idempotent,
           () -> table.putItem(item));
     } else {
@@ -461,13 +466,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
   @Override
   @Retries.OnceTranslated
-  public PathMetadata get(Path path) throws IOException {
+  public DDBPathMetadata get(Path path) throws IOException {
     return get(path, false);
   }
 
   @Override
   @Retries.OnceTranslated
-  public PathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
+  public DDBPathMetadata get(Path path, boolean wantEmptyDirectoryFlag)
       throws IOException {
     checkPath(path);
     LOG.debug("Get from table {} in region {}: {}", tableName, region, path);
@@ -485,12 +490,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @throws AmazonClientException dynamo DB level problem
    */
   @Retries.OnceRaw
-  private PathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
+  private DDBPathMetadata innerGet(Path path, boolean wantEmptyDirectoryFlag)
       throws IOException {
-    final PathMetadata meta;
+    final DDBPathMetadata meta;
     if (path.isRoot()) {
       // Root does not persist in the table
-      meta = new PathMetadata(makeDirStatus(username, path));
+      meta =
+          new DDBPathMetadata(makeDirStatus(username, path));
     } else {
       final Item item = getConsistentItem(pathToKey(path));
       meta = itemToPathMetadata(item, username);
@@ -550,15 +556,22 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
           final List<PathMetadata> metas = new ArrayList<>();
           for (Item item : items) {
-            PathMetadata meta = itemToPathMetadata(item, username);
+            DDBPathMetadata meta = itemToPathMetadata(item, username);
             metas.add(meta);
           }
+
+          DDBPathMetadata dirPathMeta = get(path);
+          boolean isAuthoritative = false;
+          if(dirPathMeta != null) {
+            isAuthoritative = dirPathMeta.isAuthoritativeDir();
+          }
+
           LOG.trace("Listing table {} in region {} for {} returning {}",
               tableName, region, path, metas);
 
-          return (metas.isEmpty() && get(path) == null)
+          return (metas.isEmpty() && dirPathMeta == null)
               ? null
-              : new DirListingMetadata(path, metas, false);
+              : new DirListingMetadata(path, metas, isAuthoritative);
         });
   }
 
@@ -567,24 +580,25 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * @param pathsToCreate paths to create
    * @return the full ancestry paths
    */
-  Collection<PathMetadata> completeAncestry(
-      Collection<PathMetadata> pathsToCreate) {
+  Collection<DDBPathMetadata> completeAncestry(
+      Collection<DDBPathMetadata> pathsToCreate) {
     // Key on path to allow fast lookup
-    Map<Path, PathMetadata> ancestry = new HashMap<>();
+    Map<Path, DDBPathMetadata> ancestry = new HashMap<>();
 
-    for (PathMetadata meta : pathsToCreate) {
+    for (DDBPathMetadata meta : pathsToCreate) {
       Preconditions.checkArgument(meta != null);
       Path path = meta.getFileStatus().getPath();
       if (path.isRoot()) {
         break;
       }
-      ancestry.put(path, meta);
+      ancestry.put(path, new DDBPathMetadata(meta));
       Path parent = path.getParent();
       while (!parent.isRoot() && !ancestry.containsKey(parent)) {
         LOG.debug("auto-create ancestor path {} for child path {}",
             parent, path);
         final FileStatus status = makeDirStatus(parent, username);
-        ancestry.put(parent, new PathMetadata(status, Tristate.FALSE, false));
+        ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE,
+            false));
         parent = parent.getParent();
       }
     }
@@ -611,13 +625,13 @@ public class DynamoDBMetadataStore implements MetadataStore {
     // Following code is to maintain this invariant by putting all ancestor
     // directories of the paths to create.
     // ancestor paths that are not explicitly added to paths to create
-    Collection<PathMetadata> newItems = new ArrayList<>();
+    Collection<DDBPathMetadata> newItems = new ArrayList<>();
     if (pathsToCreate != null) {
-      newItems.addAll(completeAncestry(pathsToCreate));
+      newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate)));
     }
     if (pathsToDelete != null) {
       for (Path meta : pathsToDelete) {
-        newItems.add(PathMetadata.tombstone(meta));
+        newItems.add(new DDBPathMetadata(PathMetadata.tombstone(meta)));
       }
     }
 
@@ -725,7 +739,11 @@ public class DynamoDBMetadataStore implements MetadataStore {
   @Override
   @Retries.OnceRaw
   public void put(Collection<PathMetadata> metas) throws IOException {
+    innerPut(pathMetaToDDBPathMeta(metas));
+  }
 
+  @Retries.OnceRaw
+  private void innerPut(Collection<DDBPathMetadata> metas) throws IOException {
     Item[] items = pathMetadataToItem(completeAncestry(metas));
     LOG.debug("Saving batch of {} items to table {}, region {}", items.length,
         tableName, region);
@@ -736,10 +754,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * Helper method to get full path of ancestors that are nonexistent in table.
    */
   @Retries.OnceRaw
-  private Collection<PathMetadata> fullPathsToPut(PathMetadata meta)
+  private Collection<DDBPathMetadata> fullPathsToPut(DDBPathMetadata meta)
       throws IOException {
     checkPathMetadata(meta);
-    final Collection<PathMetadata> metasToPut = new ArrayList<>();
+    final Collection<DDBPathMetadata> metasToPut = new ArrayList<>();
     // root path is not persisted
     if (!meta.getFileStatus().getPath().isRoot()) {
       metasToPut.add(meta);
@@ -752,7 +770,8 @@ public class DynamoDBMetadataStore implements MetadataStore {
       final Item item = getConsistentItem(pathToKey(path));
       if (!itemExists(item)) {
         final FileStatus status = makeDirStatus(path, username);
-        metasToPut.add(new PathMetadata(status, Tristate.FALSE, false));
+        metasToPut.add(new DDBPathMetadata(status, Tristate.FALSE, false,
+            meta.isAuthoritativeDir()));
         path = path.getParent();
       } else {
         break;
@@ -793,16 +812,17 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
     // directory path
     Path path = meta.getPath();
-    PathMetadata p = new PathMetadata(makeDirStatus(path, username),
-        meta.isEmpty(), false);
+    DDBPathMetadata ddbPathMeta =
+        new DDBPathMetadata(makeDirStatus(path, username), meta.isEmpty(),
+            false, meta.isAuthoritative());
 
     // First add any missing ancestors...
-    final Collection<PathMetadata> metasToPut = invoker.retry(
+    final Collection<DDBPathMetadata> metasToPut = invoker.retry(
         "paths to put", path.toString(), true,
-        () -> fullPathsToPut(p));
+        () -> fullPathsToPut(ddbPathMeta));
 
     // next add all children of the directory
-    metasToPut.addAll(meta.getListing());
+    metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing()));
 
     Invoker.once("put", path.toString(),
         () -> processBatchWriteRequest(null, pathMetadataToItem(metasToPut)));
@@ -880,21 +900,38 @@ public class DynamoDBMetadataStore implements MetadataStore {
           new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
       int delay = conf.getInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
           S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT);
+      Set<Path> parentPathSet =  new HashSet<>();
       for (Item item : expiredFiles(modTime, keyPrefix)) {
-        PathMetadata md = PathMetadataDynamoDBTranslation
+        DDBPathMetadata md = PathMetadataDynamoDBTranslation
             .itemToPathMetadata(item, username);
         Path path = md.getFileStatus().getPath();
         deletionBatch.add(path);
+
+        // add parent path of what we remove
+        Path parentPath = path.getParent();
+        if (parentPath != null) {
+          parentPathSet.add(parentPath);
+        }
+
         itemCount++;
         if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) {
           Thread.sleep(delay);
           processBatchWriteRequest(pathToKey(deletionBatch), null);
+
+          // set authoritative false for each pruned dir listing
+          removeAuthoritativeDirFlag(parentPathSet);
+          parentPathSet.clear();
+
           deletionBatch.clear();
         }
       }
       if (deletionBatch.size() > 0) {
         Thread.sleep(delay);
         processBatchWriteRequest(pathToKey(deletionBatch), null);
+
+        // set authoritative false for each pruned dir listing
+        removeAuthoritativeDirFlag(parentPathSet);
+        parentPathSet.clear();
       }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
@@ -904,6 +941,43 @@ public class DynamoDBMetadataStore implements MetadataStore {
         S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT);
   }
 
+  private void removeAuthoritativeDirFlag(Set<Path> pathSet)
+      throws IOException {
+    AtomicReference<IOException> rIOException = new AtomicReference<>();
+
+    Set<DDBPathMetadata> metas = pathSet.stream().map(path -> {
+      try {
+        DDBPathMetadata ddbPathMetadata = get(path);
+        if(ddbPathMetadata == null) {
+          return null;
+        }
+        LOG.debug("Setting false isAuthoritativeDir on {}", ddbPathMetadata);
+        ddbPathMetadata.setAuthoritativeDir(false);
+        return ddbPathMetadata;
+      } catch (IOException e) {
+        String msg = String.format("IOException while getting PathMetadata "
+            + "on path: %s.", path);
+        LOG.error(msg, e);
+        rIOException.set(e);
+        return null;
+      }
+    }).filter(Objects::nonNull).collect(Collectors.toSet());
+
+    try {
+      LOG.debug("innerPut on metas: {}", metas);
+      innerPut(metas);
+    } catch (IOException e) {
+      String msg = String.format("IOException while setting false "
+          + "authoritative directory flag on: %s.", metas);
+      LOG.error(msg, e);
+      rIOException.set(e);
+    }
+
+    if (rIOException.get() != null) {
+      throw rIOException.get();
+    }
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + '{'
@@ -1197,7 +1271,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
       map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
       map.put(TABLE, desc.toString());
       map.put(MetadataStoreCapabilities.PERSISTS_AUTHORITATIVE_BIT,
-          Boolean.toString(false));
+          Boolean.toString(true));
     } else {
       map.put("name", "DynamoDB Metadata Store");
       map.put(TABLE, "none");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
index 8515bfb..46f406f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
 
 import com.amazonaws.services.dynamodbv2.document.Item;
 import com.amazonaws.services.dynamodbv2.document.KeyAttribute;
@@ -64,6 +66,7 @@ final class PathMetadataDynamoDBTranslation {
   @VisibleForTesting
   static final String BLOCK_SIZE = "block_size";
   static final String IS_DELETED = "is_deleted";
+  static final String IS_AUTHORITATIVE = "is_authoritative";
 
   /** Table version field {@value} in version marker item. */
   @VisibleForTesting
@@ -99,12 +102,27 @@ final class PathMetadataDynamoDBTranslation {
   }
 
   /**
-   * Converts a DynamoDB item to a {@link PathMetadata}.
+   * Converts a DynamoDB item to a {@link DDBPathMetadata}.
    *
    * @param item DynamoDB item to convert
-   * @return {@code item} converted to a {@link PathMetadata}
+   * @return {@code item} converted to a {@link DDBPathMetadata}
    */
-  static PathMetadata itemToPathMetadata(Item item, String username)
+  static DDBPathMetadata itemToPathMetadata(Item item, String username)
+      throws IOException {
+    return itemToPathMetadata(item, username, false);
+  }
+
+  /**
+   * Converts a DynamoDB item to a {@link DDBPathMetadata}.
+   * Can ignore {@code IS_AUTHORITATIVE} flag if {@code ignoreIsAuthFlag} is
+   * true.
+   *
+   * @param item DynamoDB item to convert
+   * @param ignoreIsAuthFlag if true, ignore the authoritative flag on item
+   * @return {@code item} converted to a {@link DDBPathMetadata}
+   */
+  static DDBPathMetadata itemToPathMetadata(Item item, String username,
+      boolean ignoreIsAuthFlag)
       throws IOException {
     if (item == null) {
       return null;
@@ -125,8 +143,13 @@ final class PathMetadataDynamoDBTranslation {
     Path path = new Path(parent, childStr);
 
     boolean isDir = item.hasAttribute(IS_DIR) && item.getBoolean(IS_DIR);
+    boolean isAuthoritativeDir = false;
     final FileStatus fileStatus;
     if (isDir) {
+      if (!ignoreIsAuthFlag) {
+        isAuthoritativeDir = item.hasAttribute(IS_AUTHORITATIVE)
+            && item.getBoolean(IS_AUTHORITATIVE);
+      }
       fileStatus = DynamoDBMetadataStore.makeDirStatus(path, username);
     } else {
       long len = item.hasAttribute(FILE_LENGTH) ? item.getLong(FILE_LENGTH) : 0;
@@ -138,21 +161,40 @@ final class PathMetadataDynamoDBTranslation {
     boolean isDeleted =
         item.hasAttribute(IS_DELETED) && item.getBoolean(IS_DELETED);
 
-    return new PathMetadata(fileStatus, Tristate.UNKNOWN, isDeleted);
+    return new DDBPathMetadata(fileStatus, Tristate.UNKNOWN, isDeleted,
+        isAuthoritativeDir);
+  }
+
+  /**
+   * Converts a {@link DDBPathMetadata} to a DynamoDB item.
+   *
+   * @param meta {@link DDBPathMetadata} to convert
+   * @return {@code meta} converted to DynamoDB item
+   */
+  static Item pathMetadataToItem(DDBPathMetadata meta) {
+    return pathMetadataToItem(meta, false);
   }
 
   /**
-   * Converts a {@link PathMetadata} to a DynamoDB item.
+   * Converts a {@link DDBPathMetadata} to a DynamoDB item.
+   *
+   * Can ignore {@code IS_AUTHORITATIVE} flag if {@code ignoreIsAuthFlag} is
+   * true.
    *
-   * @param meta {@link PathMetadata} to convert
+   * @param meta {@link DDBPathMetadata} to convert
+   * @param ignoreIsAuthFlag if true, ignore the authoritative flag on item
    * @return {@code meta} converted to DynamoDB item
    */
-  static Item pathMetadataToItem(PathMetadata meta) {
+  static Item pathMetadataToItem(DDBPathMetadata meta,
+      boolean ignoreIsAuthFlag) {
     Preconditions.checkNotNull(meta);
     final FileStatus status = meta.getFileStatus();
     final Item item = new Item().withPrimaryKey(pathToKey(status.getPath()));
     if (status.isDirectory()) {
       item.withBoolean(IS_DIR, true);
+      if (!ignoreIsAuthFlag) {
+        item.withBoolean(IS_AUTHORITATIVE, meta.isAuthoritativeDir());
+      }
     } else {
       item.withLong(FILE_LENGTH, status.getLen())
           .withLong(MOD_TIME, status.getModificationTime())
@@ -214,18 +256,19 @@ final class PathMetadataDynamoDBTranslation {
   }
 
   /**
-   * Converts a collection {@link PathMetadata} to a collection DynamoDB items.
+   * Converts a collection {@link DDBPathMetadata} to a collection DynamoDB
+   * items.
    *
-   * @see #pathMetadataToItem(PathMetadata)
+   * @see #pathMetadataToItem(DDBPathMetadata)
    */
-  static Item[] pathMetadataToItem(Collection<PathMetadata> metas) {
+  static Item[] pathMetadataToItem(Collection<DDBPathMetadata> metas) {
     if (metas == null) {
       return null;
     }
 
     final Item[] items = new Item[metas.size()];
     int i = 0;
-    for (PathMetadata meta : metas) {
+    for (DDBPathMetadata meta : metas) {
       items[i++] = pathMetadataToItem(meta);
     }
     return items;
@@ -301,4 +344,10 @@ final class PathMetadataDynamoDBTranslation {
   private PathMetadataDynamoDBTranslation() {
   }
 
+  static List<DDBPathMetadata> pathMetaToDDBPathMeta(
+      Collection<PathMetadata> pathMetadatas) {
+    return pathMetadatas.stream().map(p -> new DDBPathMetadata(p))
+        .collect(Collectors.toList());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
index 19cfe1b..cc55951 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
@@ -235,6 +235,10 @@ public final class S3Guard {
       changed = changed || updated;
     }
 
+    // If dirMeta is not authoritative, but isAuthoritative is true the
+    // directory metadata should be updated. Treat it as a change.
+    changed = changed || (!dirMeta.isAuthoritative() && isAuthoritative);
+
     if (changed && isAuthoritative) {
       dirMeta.setAuthoritative(true); // This is the full directory contents
       ms.put(dirMeta);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
index 66ee11d..b86d275 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -122,8 +122,9 @@ two different reasons:
     (`DirListingMetadata`) is full, and complete.
     * If set to `FALSE` the listing may not be complete.
     * Metadata store may persist the isAuthoritative bit on the metadata store.
-    * Currently only `org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore`
-    implementation supports authoritative bit.
+    * Currently `org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore` and
+    `org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore` implementation
+    supports authoritative bit.
 
 More on Authoritative S3Guard:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
index 5a59400..45d6051 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java
@@ -727,6 +727,13 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
         new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
         Tristate.FALSE, false));
 
+    // set parent dir as authoritative
+    if (!allowMissing()) {
+      DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir));
+      parentDirMd.setAuthoritative(true);
+      ms.put(parentDirMd);
+    }
+
     ms.prune(time);
     DirListingMetadata listing;
     for (String directory : directories) {
@@ -739,6 +746,48 @@ public abstract class MetadataStoreTestBase extends HadoopTestBase {
   }
 
   @Test
+  public void testPrunePreservesAuthoritative() throws Exception {
+    String rootDir = "/unpruned-root-dir";
+    String grandparentDir = rootDir + "/pruned-grandparent-dir";
+    String parentDir = grandparentDir + "/pruned-parent-dir";
+    String staleFile = parentDir + "/stale-file";
+    String freshFile = rootDir + "/fresh-file";
+    String[] directories = {rootDir, grandparentDir, parentDir};
+
+    // create dirs
+    createNewDirs(rootDir, grandparentDir, parentDir);
+    long time = System.currentTimeMillis();
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time + 1, strToPath(staleFile)),
+        Tristate.FALSE, false));
+    ms.put(new PathMetadata(
+        new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)),
+        Tristate.FALSE, false));
+
+    if (!allowMissing()) {
+      // set parent dir as authoritative
+      DirListingMetadata parentDirMd = ms.listChildren(strToPath(parentDir));
+      parentDirMd.setAuthoritative(true);
+      ms.put(parentDirMd);
+
+      // prune the ms
+      ms.prune(time);
+
+      // get the directory listings
+      DirListingMetadata rootDirMd = ms.listChildren(strToPath(rootDir));
+      DirListingMetadata grandParentDirMd =
+          ms.listChildren(strToPath(grandparentDir));
+      parentDirMd = ms.listChildren(strToPath(parentDir));
+
+      // assert that parent dir is still authoritative (no removed elements
+      // during prune)
+      assertFalse(rootDirMd.isAuthoritative());
+      assertFalse(grandParentDirMd.isAuthoritative());
+      assertTrue(parentDirMd.isAuthoritative());
+    }
+  }
+
+  @Test
   public void testPutDirListingMetadataPutsFileMetadata()
       throws IOException {
     boolean authoritative = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7232857/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
index 1678746..70d4c3b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.LambdaTestUtils;
+import org.mockito.Mockito;
 
 import static com.amazonaws.services.dynamodbv2.model.KeyType.HASH;
 import static com.amazonaws.services.dynamodbv2.model.KeyType.RANGE;
@@ -50,6 +51,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*;
 import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER;
 import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION;
+import static org.mockito.Mockito.never;
 
 /**
  * Test the PathMetadataDynamoDBTranslation is able to translate between domain
@@ -59,28 +61,30 @@ public class TestPathMetadataDynamoDBTranslation extends Assert {
 
   private static final Path TEST_DIR_PATH = new Path("s3a://test-bucket/myDir");
   private static final Item TEST_DIR_ITEM = new Item();
-  private static PathMetadata testDirPathMetadata;
+  private static DDBPathMetadata testDirPathMetadata;
 
   private static final long TEST_FILE_LENGTH = 100;
   private static final long TEST_MOD_TIME = 9999;
   private static final long TEST_BLOCK_SIZE = 128;
   private static final Path TEST_FILE_PATH = new Path(TEST_DIR_PATH, "myFile");
   private static final Item TEST_FILE_ITEM = new Item();
-  private static PathMetadata testFilePathMetadata;
+  private static DDBPathMetadata testFilePathMetadata;
 
   @BeforeClass
   public static void setUpBeforeClass() throws IOException {
     String username = UserGroupInformation.getCurrentUser().getShortUserName();
 
-    testDirPathMetadata =
-        new PathMetadata(new S3AFileStatus(false, TEST_DIR_PATH, username));
+    testDirPathMetadata = new DDBPathMetadata(new S3AFileStatus(false,
+        TEST_DIR_PATH, username));
+
     TEST_DIR_ITEM
         .withPrimaryKey(PARENT, "/test-bucket", CHILD, TEST_DIR_PATH.getName())
         .withBoolean(IS_DIR, true);
 
-    testFilePathMetadata = new PathMetadata(
+    testFilePathMetadata = new DDBPathMetadata(
         new S3AFileStatus(TEST_FILE_LENGTH, TEST_MOD_TIME, TEST_FILE_PATH,
             TEST_BLOCK_SIZE, username));
+
     TEST_FILE_ITEM
         .withPrimaryKey(PARENT, pathToParentKey(TEST_FILE_PATH.getParent()),
             CHILD, TEST_FILE_PATH.getName())
@@ -235,4 +239,37 @@ public class TestPathMetadataDynamoDBTranslation extends Assert {
         itemToPathMetadata(marker, "alice"));
   }
 
+  /**
+   * Test when translating an {@link Item} to {@link DDBPathMetadata} works
+   * if {@code IS_AUTHORITATIVE} flag is ignored.
+   */
+  @Test
+  public void testIsAuthoritativeCompatibilityItemToPathMetadata()
+      throws Exception {
+    Item item = Mockito.spy(TEST_DIR_ITEM);
+    item.withBoolean(IS_AUTHORITATIVE, true);
+
+    final String user =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    DDBPathMetadata meta = itemToPathMetadata(item, user, true);
+
+    Mockito.verify(item, Mockito.never()).getBoolean(IS_AUTHORITATIVE);
+    assertFalse(meta.isAuthoritativeDir());
+  }
+
+  /**
+   * Test when translating an {@link DDBPathMetadata} to {@link Item} works
+   * if {@code IS_AUTHORITATIVE} flag is ignored.
+   */
+  @Test
+  public void testIsAuthoritativeCompatibilityPathMetadataToItem() {
+    DDBPathMetadata meta = Mockito.spy(testFilePathMetadata);
+    meta.setAuthoritativeDir(true);
+
+    Item item = pathMetadataToItem(meta, true);
+
+    Mockito.verify(meta, never()).isAuthoritativeDir();
+    assertFalse(item.hasAttribute(IS_AUTHORITATIVE));
+  }
+
 }


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


[32/50] [abbrv] hadoop git commit: HDDS-350. ContainerMapping#flushContainerInfo doesn't set containerId. Contributed by Ajay Kumar.

Posted by sh...@apache.org.
HDDS-350. ContainerMapping#flushContainerInfo doesn't set containerId. Contributed by Ajay Kumar.


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

Branch: refs/heads/HDFS-12943
Commit: 4c25f37c6cc4e22a006cd095d6143b549bf4a0a8
Parents: 5aa15cf
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Wed Aug 22 10:53:54 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Wed Aug 22 10:54:10 2018 -0700

----------------------------------------------------------------------
 .../scm/container/common/helpers/ContainerInfo.java |  1 +
 .../hadoop/hdds/scm/container/ContainerMapping.java | 16 +---------------
 .../hdds/scm/container/TestContainerMapping.java    |  8 ++++++++
 3 files changed, 10 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
index 427c08b..311c118 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
@@ -212,6 +212,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
   public HddsProtos.SCMContainerInfo getProtobuf() {
     HddsProtos.SCMContainerInfo.Builder builder =
         HddsProtos.SCMContainerInfo.newBuilder();
+    Preconditions.checkState(containerID > 0);
     return builder.setAllocatedBytes(getAllocatedBytes())
         .setContainerID(getContainerID())
         .setUsedBytes(getUsedBytes())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index 11863f2..4076dad 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -733,21 +733,7 @@ public class ContainerMapping implements Mapping {
         // return info of a deleted container. may revisit this in the future,
         // for now, just skip a not-found container
         if (containerBytes != null) {
-          HddsProtos.SCMContainerInfo oldInfoProto =
-              HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
-          ContainerInfo oldInfo = ContainerInfo.fromProtobuf(oldInfoProto);
-          ContainerInfo newInfo = new ContainerInfo.Builder()
-              .setAllocatedBytes(info.getAllocatedBytes())
-              .setNumberOfKeys(oldInfo.getNumberOfKeys())
-              .setOwner(oldInfo.getOwner())
-              .setPipelineID(oldInfo.getPipelineID())
-              .setState(oldInfo.getState())
-              .setUsedBytes(oldInfo.getUsedBytes())
-              .setDeleteTransactionId(oldInfo.getDeleteTransactionId())
-              .setReplicationFactor(oldInfo.getReplicationFactor())
-              .setReplicationType(oldInfo.getReplicationType())
-              .build();
-          containerStore.put(dbKey, newInfo.getProtobuf().toByteArray());
+          containerStore.put(dbKey, info.getProtobuf().toByteArray());
         } else {
           LOG.debug("Container state manager has container {} but not found " +
                   "in container store, a deleted container?",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c25f37c/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
index 2dc7e99..1e9c35b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
@@ -360,4 +360,12 @@ public class TestContainerMapping {
     return containerInfo;
   }
 
+  @Test
+  public void testFlushAllContainers() throws IOException {
+    ContainerInfo info = createContainer();
+    List<ContainerInfo> containers = mapping.getStateManager().getAllContainers();
+    Assert.assertTrue(containers.size() > 0);
+    mapping.flushContainerInfo();
+  }
+
 }


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


[39/50] [abbrv] hadoop git commit: HDFS-13805. Journal Nodes should allow to format non-empty directories with -force option. Contributed by Surendra Singh Lilhore.

Posted by sh...@apache.org.
HDFS-13805. Journal Nodes should allow to format non-empty directories with -force option. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/HDFS-12943
Commit: 96c4575d7373079becfa3e3db29ba98e6fb86388
Parents: ca29fb7
Author: Surendra Singh Lilhore <su...@apache.org>
Authored: Fri Aug 24 08:14:57 2018 +0530
Committer: Surendra Singh Lilhore <su...@apache.org>
Committed: Fri Aug 24 08:14:57 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/qjournal/client/AsyncLogger.java   |  3 ++-
 .../hdfs/qjournal/client/AsyncLoggerSet.java       |  4 ++--
 .../hdfs/qjournal/client/IPCLoggerChannel.java     |  5 +++--
 .../hdfs/qjournal/client/QuorumJournalManager.java |  4 ++--
 .../hdfs/qjournal/protocol/QJournalProtocol.java   |  2 +-
 .../QJournalProtocolServerSideTranslatorPB.java    |  2 +-
 .../protocolPB/QJournalProtocolTranslatorPB.java   |  6 ++++--
 .../hadoop/hdfs/qjournal/server/JNStorage.java     |  4 ++--
 .../hadoop/hdfs/qjournal/server/Journal.java       |  6 +++---
 .../hdfs/qjournal/server/JournalNodeRpcServer.java |  5 +++--
 .../hdfs/server/namenode/BackupJournalManager.java |  2 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java     |  5 +++--
 .../hadoop/hdfs/server/namenode/FSImage.java       |  5 +++--
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  4 ++--
 .../hdfs/server/namenode/FileJournalManager.java   |  2 +-
 .../hdfs/server/namenode/JournalManager.java       |  2 +-
 .../hadoop/hdfs/server/namenode/JournalSet.java    |  2 +-
 .../hadoop/hdfs/server/namenode/NameNode.java      |  4 ++--
 .../src/main/proto/QJournalProtocol.proto          |  1 +
 .../hdfs/qjournal/client/TestEpochsAreUnique.java  |  2 +-
 .../hdfs/qjournal/client/TestQJMWithFaults.java    |  6 +++---
 .../qjournal/client/TestQuorumJournalManager.java  |  4 ++--
 .../client/TestQuorumJournalManagerUnit.java       |  4 +++-
 .../hadoop/hdfs/qjournal/server/TestJournal.java   | 17 ++++++++++++++---
 .../hdfs/qjournal/server/TestJournalNode.java      |  4 ++--
 .../qjournal/server/TestJournalNodeMXBean.java     |  2 +-
 .../hdfs/qjournal/server/TestJournalNodeSync.java  |  2 +-
 .../server/namenode/TestGenericJournalConf.java    |  2 +-
 .../tools/contract/AbstractContractDistCpTest.java |  2 +-
 29 files changed, 67 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
index d2b48cc..2633723 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
@@ -89,8 +89,9 @@ interface AsyncLogger {
   /**
    * Format the log directory.
    * @param nsInfo the namespace info to format with
+   * @param force the force option to format
    */
-  public ListenableFuture<Void> format(NamespaceInfo nsInfo);
+  public ListenableFuture<Void> format(NamespaceInfo nsInfo, boolean force);
 
   /**
    * @return whether or not the remote node has any valid data.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
index d46c2cf..b52e312 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
@@ -299,12 +299,12 @@ class AsyncLoggerSet {
     return QuorumCall.create(calls);
   }
 
-  QuorumCall<AsyncLogger,Void> format(NamespaceInfo nsInfo) {
+  QuorumCall<AsyncLogger, Void> format(NamespaceInfo nsInfo, boolean force) {
     Map<AsyncLogger, ListenableFuture<Void>> calls =
         Maps.newHashMap();
     for (AsyncLogger logger : loggers) {
       ListenableFuture<Void> future =
-          logger.format(nsInfo);
+          logger.format(nsInfo, force);
       calls.put(logger, future);
     }
     return QuorumCall.create(calls);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
index 3036735..4fca1bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
@@ -502,11 +502,12 @@ public class IPCLoggerChannel implements AsyncLogger {
   }
 
   @Override
-  public ListenableFuture<Void> format(final NamespaceInfo nsInfo) {
+  public ListenableFuture<Void> format(final NamespaceInfo nsInfo,
+      final boolean force) {
     return singleThreadExecutor.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        getProxy().format(journalId, nameServiceId, nsInfo);
+        getProxy().format(journalId, nameServiceId, nsInfo, force);
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
index 4faaa98..bd45292 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@ -220,8 +220,8 @@ public class QuorumJournalManager implements JournalManager {
   }
   
   @Override
-  public void format(NamespaceInfo nsInfo) throws IOException {
-    QuorumCall<AsyncLogger,Void> call = loggers.format(nsInfo);
+  public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
+    QuorumCall<AsyncLogger, Void> call = loggers.format(nsInfo, force);
     try {
       call.waitFor(loggers.size(), loggers.size(), 0, timeoutMs,
           "format");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
index 5558bd5..8dad261 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
@@ -68,7 +68,7 @@ public interface QJournalProtocol {
    * Format the underlying storage for the given namespace.
    */
   void format(String journalId, String nameServiceId,
-      NamespaceInfo nsInfo) throws IOException;
+      NamespaceInfo nsInfo, boolean force) throws IOException;
 
   /**
    * Begin a new epoch. See the HDFS-3077 design doc for details.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
index 865d296..2ad19da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
@@ -147,7 +147,7 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
     try {
       impl.format(request.getJid().getIdentifier(),
           request.hasNameServiceId() ? request.getNameServiceId() : null,
-          PBHelper.convert(request.getNsInfo()));
+          PBHelper.convert(request.getNsInfo()), request.getForce());
       return FormatResponseProto.getDefaultInstance();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
index d7cd7b5..42d35f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
@@ -136,11 +136,13 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
   @Override
   public void format(String jid,
                      String nameServiceId,
-                     NamespaceInfo nsInfo) throws IOException {
+                     NamespaceInfo nsInfo,
+                     boolean force) throws IOException {
     try {
       FormatRequestProto.Builder req = FormatRequestProto.newBuilder()
           .setJid(convertJournalId(jid))
-          .setNsInfo(PBHelper.convert(nsInfo));
+          .setNsInfo(PBHelper.convert(nsInfo))
+          .setForce(force);
       if(nameServiceId != null) {
         req.setNameServiceId(nameServiceId);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
index 6bf4903..612fd3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
@@ -204,10 +204,10 @@ class JNStorage extends Storage {
     }
   }
 
-  void format(NamespaceInfo nsInfo) throws IOException {
+  void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     unlockAll();
     try {
-      sd.analyzeStorage(StartupOption.FORMAT, this, true);
+      sd.analyzeStorage(StartupOption.FORMAT, this, !force);
     } finally {
       sd.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 8f25d26..7e88afa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -227,13 +227,13 @@ public class Journal implements Closeable {
   /**
    * Format the local storage with the given namespace.
    */
-  void format(NamespaceInfo nsInfo) throws IOException {
+  void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     Preconditions.checkState(nsInfo.getNamespaceID() != 0,
         "can't format with uninitialized namespace info: %s",
         nsInfo);
     LOG.info("Formatting journal id : " + journalId + " with namespace info: " +
-        nsInfo);
-    storage.format(nsInfo);
+        nsInfo + " and force: " + force);
+    storage.format(nsInfo, force);
     refreshCachedData();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
index b1a3c96..0f11026 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
@@ -176,9 +176,10 @@ public class JournalNodeRpcServer implements QJournalProtocol,
   @Override
   public void format(String journalId,
                      String nameServiceId,
-                     NamespaceInfo nsInfo)
+                     NamespaceInfo nsInfo,
+                     boolean force)
       throws IOException {
-    jn.getOrCreateJournal(journalId, nameServiceId).format(nsInfo);
+    jn.getOrCreateJournal(journalId, nameServiceId).format(nsInfo, force);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
index e1ddfb9..eac91bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
@@ -42,7 +42,7 @@ class BackupJournalManager implements JournalManager {
   }
 
   @Override
-  public void format(NamespaceInfo nsInfo) {
+  public void format(NamespaceInfo nsInfo, boolean force) {
     // format() should only get called at startup, before any BNs
     // can register with the NN.
     throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index d6fb212..547ad57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -417,13 +417,14 @@ public class FSEditLog implements LogsPurgeable {
    * File-based journals are skipped, since they are formatted by the
    * Storage format code.
    */
-  synchronized void formatNonFileJournals(NamespaceInfo nsInfo) throws IOException {
+  synchronized void formatNonFileJournals(NamespaceInfo nsInfo, boolean force)
+      throws IOException {
     Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
         "Bad state: %s", state);
     
     for (JournalManager jm : journalSet.getJournalManagers()) {
       if (!(jm instanceof FileJournalManager)) {
-        jm.format(nsInfo);
+        jm.format(nsInfo, force);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 5cfc017..6d107be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -160,7 +160,8 @@ public class FSImage implements Closeable {
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
  
-  void format(FSNamesystem fsn, String clusterId) throws IOException {
+  void format(FSNamesystem fsn, String clusterId, boolean force)
+      throws IOException {
     long fileCount = fsn.getFilesTotal();
     // Expect 1 file, which is the root inode
     Preconditions.checkState(fileCount == 1,
@@ -171,7 +172,7 @@ public class FSImage implements Closeable {
     ns.clusterID = clusterId;
     
     storage.format(ns);
-    editLog.formatNonFileJournals(ns);
+    editLog.formatNonFileJournals(ns, force);
     saveFSImageInAllDirs(fsn, 0);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b0fb26c..06bf008 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1078,8 +1078,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // format before starting up if requested
     if (startOpt == StartupOption.FORMAT) {
-      
-      fsImage.format(this, fsImage.getStorage().determineClusterId());// reuse current id
+      // reuse current id
+      fsImage.format(this, fsImage.getStorage().determineClusterId(), false);
 
       startOpt = StartupOption.REGULAR;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index c71c09a..185ad73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -100,7 +100,7 @@ public class FileJournalManager implements JournalManager {
   public void close() throws IOException {}
   
   @Override
-  public void format(NamespaceInfo ns) throws IOException {
+  public void format(NamespaceInfo ns, boolean force) throws IOException {
     // Formatting file journals is done by the StorageDirectory
     // format code, since they may share their directory with
     // checkpoints, etc.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
index ae1bc3b..d6d2094 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
@@ -43,7 +43,7 @@ public interface JournalManager extends Closeable, FormatConfirmable,
    * Format the underlying storage, removing any previously
    * stored data.
    */
-  void format(NamespaceInfo ns) throws IOException;
+  void format(NamespaceInfo ns, boolean force) throws IOException;
 
   /**
    * Begin writing to a new segment of the log stream, which starts at

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
index e7f2adb..868df01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
@@ -188,7 +188,7 @@ public class JournalSet implements JournalManager {
   }
   
   @Override
-  public void format(NamespaceInfo nsInfo) throws IOException {
+  public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
     // The operation is done by FSEditLog itself
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 7f78d2f..a8034da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1159,7 +1159,7 @@ public class NameNode extends ReconfigurableBase implements
         return true; // aborted
       }
 
-      fsImage.format(fsn, clusterId);
+      fsImage.format(fsn, clusterId, force);
     } catch (IOException ioe) {
       LOG.warn("Encountered exception during format: ", ioe);
       fsImage.close();
@@ -1262,7 +1262,7 @@ public class NameNode extends ReconfigurableBase implements
       // actually want to save a checkpoint - just prime the dirs with
       // the existing namespace info
       newSharedStorage.format(nsInfo);
-      sharedEditsImage.getEditLog().formatNonFileJournals(nsInfo);
+      sharedEditsImage.getEditLog().formatNonFileJournals(nsInfo, force);
 
       // Need to make sure the edit log segments are in good shape to initialize
       // the shared edits dir.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
index a37c723..625966f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
@@ -242,6 +242,7 @@ message FormatRequestProto {
   required JournalIdProto jid = 1;
   required NamespaceInfoProto nsInfo = 2;
   optional string nameServiceId = 3;
+  optional bool force = 4 [ default = false ];
 }
 
 message FormatResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
index 5101a41..0fc1429 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
@@ -56,7 +56,7 @@ public class TestEpochsAreUnique {
     QuorumJournalManager qjm = new QuorumJournalManager(
         conf, uri, FAKE_NSINFO);
     try {
-      qjm.format(FAKE_NSINFO);
+      qjm.format(FAKE_NSINFO, false);
     } finally {
       qjm.close();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
index 6ad43f5..40f213e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java
@@ -105,7 +105,7 @@ public class TestQJMWithFaults {
     long ret;
     try {
       qjm = createInjectableQJM(cluster);
-      qjm.format(FAKE_NSINFO);
+      qjm.format(FAKE_NSINFO, false);
       doWorkload(cluster, qjm);
       
       SortedSet<Integer> ipcCounts = Sets.newTreeSet();
@@ -156,7 +156,7 @@ public class TestQJMWithFaults {
         QuorumJournalManager qjm = null;
         try {
           qjm = createInjectableQJM(cluster);
-          qjm.format(FAKE_NSINFO);
+          qjm.format(FAKE_NSINFO, false);
           List<AsyncLogger> loggers = qjm.getLoggerSetForTests().getLoggersForTests();
           failIpcNumber(loggers.get(0), failA);
           failIpcNumber(loggers.get(1), failB);
@@ -240,7 +240,7 @@ public class TestQJMWithFaults {
     // Format the cluster using a non-faulty QJM.
     QuorumJournalManager qjmForInitialFormat =
         createInjectableQJM(cluster);
-    qjmForInitialFormat.format(FAKE_NSINFO);
+    qjmForInitialFormat.format(FAKE_NSINFO, false);
     qjmForInitialFormat.close();
     
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
index 69856ae..00bec22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
@@ -100,7 +100,7 @@ public class TestQuorumJournalManager {
     qjm = createSpyingQJM();
     spies = qjm.getLoggerSetForTests().getLoggersForTests();
 
-    qjm.format(QJMTestUtil.FAKE_NSINFO);
+    qjm.format(QJMTestUtil.FAKE_NSINFO, false);
     qjm.recoverUnfinalizedSegments();
     assertEquals(1, qjm.getLoggerSetForTests().getEpoch());
   }
@@ -149,7 +149,7 @@ public class TestQuorumJournalManager {
     QuorumJournalManager qjm = closeLater(new QuorumJournalManager(
         conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO));
     assertFalse(qjm.hasSomeData());
-    qjm.format(FAKE_NSINFO);
+    qjm.format(FAKE_NSINFO, false);
     assertTrue(qjm.hasSomeData());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
index 75dcf2f..9e1e3bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.qjournal.client;
 
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
@@ -89,7 +90,8 @@ public class TestQuorumJournalManagerUnit {
           NewEpochResponseProto.newBuilder().build()
           ).when(logger).newEpoch(Mockito.anyLong());
       
-      futureReturns(null).when(logger).format(Mockito.<NamespaceInfo>any());
+      futureReturns(null).when(logger).format(Mockito.<NamespaceInfo>any(),
+          anyBoolean());
     }
     
     qjm.recoverUnfinalizedSegments();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
index b71d694..b8d2652 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
@@ -73,7 +73,7 @@ public class TestJournal {
     conf = new Configuration();
     journal = new Journal(conf, TEST_LOG_DIR, JID, StartupOption.REGULAR,
       mockErrorReporter);
-    journal.format(FAKE_NSINFO);
+    journal.format(FAKE_NSINFO, false);
   }
   
   @After
@@ -207,7 +207,7 @@ public class TestJournal {
     // Clear the storage directory before reformatting it
     journal.getStorage().getJournalManager()
         .getStorageDirectory().clearDirectory();
-    journal.format(FAKE_NSINFO_2);
+    journal.format(FAKE_NSINFO_2, false);
     
     assertEquals(0, journal.getLastPromisedEpoch());
     assertEquals(0, journal.getLastWriterEpoch());
@@ -425,7 +425,7 @@ public class TestJournal {
     try {
       // Format again here and to format the non-empty directories in
       // journal node.
-      journal.format(FAKE_NSINFO);
+      journal.format(FAKE_NSINFO, false);
       fail("Did not fail to format non-empty directories in journal node.");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
@@ -434,4 +434,15 @@ public class TestJournal {
     }
   }
 
+  @Test
+  public void testFormatNonEmptyStorageDirectoriesWhenforceOptionIsTrue()
+      throws Exception {
+    try {
+      // Format again here and to format the non-empty directories in
+      // journal node.
+      journal.format(FAKE_NSINFO, true);
+    } catch (IOException ioe) {
+      fail("Format should be success with force option.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
index 8d58792..4cc5968 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
@@ -159,11 +159,11 @@ public class TestJournalNode {
             HdfsServerConstants.StartupOption.REGULAR);
         NamespaceInfo fakeNameSpaceInfo = new NamespaceInfo(
             12345, "mycluster", "my-bp"+nsId, 0L);
-        journal.format(fakeNameSpaceInfo);
+        journal.format(fakeNameSpaceInfo, false);
       }
     } else {
       journal = jn.getOrCreateJournal(journalId);
-      journal.format(FAKE_NSINFO);
+      journal.format(FAKE_NSINFO, false);
     }
 
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
index 1de37a4..7550c4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeMXBean.java
@@ -82,7 +82,7 @@ public class TestJournalNodeMXBean {
     // format the journal ns1
     final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(12345, "mycluster",
         "my-bp", 0L);
-    jn.getOrCreateJournal(NAMESERVICE).format(FAKE_NSINFO);
+    jn.getOrCreateJournal(NAMESERVICE).format(FAKE_NSINFO, false);
 
     // check again after format
     // getJournalsStatus

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
index 8de9641..c23604b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java
@@ -341,7 +341,7 @@ public class TestJournalNodeSync {
     }
 
     // Format the JN
-    journal1.format(nsInfo);
+    journal1.format(nsInfo, false);
 
     // Roll some more edits
     for (int i = 4; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
index 020ecb5..edcf9e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
@@ -155,7 +155,7 @@ public class TestGenericJournalConf {
     }
     
     @Override
-    public void format(NamespaceInfo nsInfo) throws IOException {
+    public void format(NamespaceInfo nsInfo, boolean force) throws IOException {
       formatCalled = true;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96c4575d/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
index 5c74430..0757a66 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java
@@ -249,7 +249,7 @@ public abstract class AbstractContractDistCpTest
     Counter c = job.getCounters().findCounter(counter);
     long value = c.getValue();
     String description =
-        String.format("%s value %s", c.getDisplayName(), value);
+        String.format("%s value %s", c.getDisplayName(), value, false);
 
     if (min >= 0) {
       assertTrue(description + " too below minimum " + min,


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


[13/50] [abbrv] hadoop git commit: HADOOP-14624. Add GenericTestUtils.DelayAnswer that accept slf4j logger API. Contributed by Ian Pickering and Wenxin He.

Posted by sh...@apache.org.
HADOOP-14624. Add GenericTestUtils.DelayAnswer that accept slf4j logger API. Contributed by Ian Pickering and Wenxin He.


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

Branch: refs/heads/HDFS-12943
Commit: 79c97f6a0bebc95ff81a8ef9b07d3619f05ed583
Parents: ab37423
Author: Giovanni Matteo Fumarola <gi...@apache.com>
Authored: Fri Aug 17 14:40:00 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.com>
Committed: Fri Aug 17 14:40:00 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/test/GenericTestUtils.java  |  8 ++++----
 .../org/apache/hadoop/hdfs/TestDFSClientFailover.java  |  6 ++++--
 .../java/org/apache/hadoop/hdfs/TestFileAppend4.java   |  6 +++---
 .../java/org/apache/hadoop/hdfs/TestReplication.java   |  6 ++++--
 .../hdfs/qjournal/client/TestIPCLoggerChannel.java     |  6 +++---
 .../hdfs/server/datanode/BlockReportTestBase.java      |  8 ++++----
 .../hadoop/hdfs/server/namenode/FSImageTestUtil.java   | 13 +++++++++++++
 .../hadoop/hdfs/server/namenode/TestCheckpoint.java    |  4 +++-
 .../hadoop/hdfs/server/namenode/TestDeleteRace.java    | 10 +++++-----
 .../hadoop/hdfs/server/namenode/TestSaveNamespace.java | 10 +++++-----
 .../hadoop/hdfs/server/namenode/ha/TestDNFencing.java  |  6 +++---
 .../hdfs/server/namenode/ha/TestPipelinesFailover.java | 10 +++++-----
 .../server/namenode/ha/TestStandbyCheckpoints.java     |  6 +++---
 13 files changed, 59 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 0112894..d68f4e2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -501,7 +501,7 @@ public abstract class GenericTestUtils {
    * method is called, then waits on another before continuing.
    */
   public static class DelayAnswer implements Answer<Object> {
-    private final Log LOG;
+    private final org.slf4j.Logger LOG;
 
     private final CountDownLatch fireLatch = new CountDownLatch(1);
     private final CountDownLatch waitLatch = new CountDownLatch(1);
@@ -514,7 +514,7 @@ public abstract class GenericTestUtils {
     private volatile Throwable thrown;
     private volatile Object returnValue;
 
-    public DelayAnswer(Log log) {
+    public DelayAnswer(org.slf4j.Logger log) {
       this.LOG = log;
     }
 
@@ -611,13 +611,13 @@ public abstract class GenericTestUtils {
    */
   public static class DelegateAnswer implements Answer<Object> {
     private final Object delegate;
-    private final Log log;
+    private final org.slf4j.Logger log;
 
     public DelegateAnswer(Object delegate) {
       this(null, delegate);
     }
 
-    public DelegateAnswer(Log log, Object delegate) {
+    public DelegateAnswer(org.slf4j.Logger log, Object delegate) {
       this.log = log;
       this.delegate = delegate;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
index c14ebb4..f9d0460 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
@@ -34,6 +34,8 @@ import java.util.List;
 
 import javax.net.SocketFactory;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -67,7 +69,7 @@ import sun.net.spi.nameservice.NameService;
 
 public class TestDFSClientFailover {
   
-  private static final Log LOG = LogFactory.getLog(TestDFSClientFailover.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestDFSClientFailover.class);
   
   private static final Path TEST_FILE = new Path("/tmp/failover-test-file");
   private static final int FILE_LENGTH_TO_VERIFY = 100;
@@ -239,7 +241,7 @@ public class TestDFSClientFailover {
       List<NameService> nsList = (List<NameService>) f.get(null);
 
       NameService ns = nsList.get(0);
-      Log log = LogFactory.getLog("NameServiceSpy");
+      Logger log = LoggerFactory.getLogger("NameServiceSpy");
       
       ns = Mockito.mock(NameService.class,
           new GenericTestUtils.DelegateAnswer(log, ns));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index 40bc314..ff0fd19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -32,8 +32,8 @@ import java.io.OutputStream;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -57,7 +57,7 @@ import org.junit.Test;
  *  using append()/sync() to recover block information
  */
 public class TestFileAppend4 {
-  static final Log LOG = LogFactory.getLog(TestFileAppend4.class);
+  static final Logger LOG = LoggerFactory.getLogger(TestFileAppend4.class);
   static final long BLOCK_SIZE = 1024;
   static final long BBW_SIZE = 500; // don't align on bytes/checksum
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
index 0f1bedd..b4f0c2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
@@ -35,6 +35,8 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -78,8 +80,8 @@ public class TestReplication {
     "/d1/r1", "/d1/r1", "/d1/r2", "/d1/r2", "/d1/r2", "/d2/r3", "/d2/r3"
   };
   private static final int numDatanodes = racks.length;
-  private static final Log LOG = LogFactory.getLog(
-                                       "org.apache.hadoop.hdfs.TestReplication");
+  private static final Logger LOG = LoggerFactory.getLogger(
+                                       TestReplication.class);
   
   /* check if there are at least two nodes are on the same rack */
   private void checkFile(FileSystem fileSys, Path name, int repl)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
index ba51372..d649686 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
@@ -24,8 +24,8 @@ import java.net.InetSocketAddress;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
@@ -43,7 +43,7 @@ import org.mockito.Mockito;
 import com.google.common.base.Supplier;
 
 public class TestIPCLoggerChannel {
-  private static final Log LOG = LogFactory.getLog(
+  private static final Logger LOG = LoggerFactory.getLogger(
       TestIPCLoggerChannel.class);
   
   private final Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index c9ff572..535b629 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -38,8 +38,8 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -87,7 +87,7 @@ import org.mockito.invocation.InvocationOnMock;
  * and messages.
  */
 public abstract class BlockReportTestBase {
-  public static final Log LOG = LogFactory.getLog(BlockReportTestBase.class);
+  public static final Logger LOG = LoggerFactory.getLogger(BlockReportTestBase.class);
 
   private static short REPL_FACTOR = 1;
   private static final int RAND_LIMIT = 2000;
@@ -879,7 +879,7 @@ public abstract class BlockReportTestBase {
   private static void initLoggers() {
     DFSTestUtil.setNameNodeLogLevel(Level.ALL);
     GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(BlockReportTestBase.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockReportTestBase.LOG, org.slf4j.event.Level.DEBUG);
   }
 
   private Block findBlock(Path path, long size) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index f990c5e..9f678cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -574,6 +574,19 @@ public abstract class FSImageTestUtil {
       }
     }
   }
+
+  public static void logStorageContents(org.slf4j.Logger LOG, NNStorage storage) {
+    LOG.info("current storages and corresponding sizes:");
+    for (StorageDirectory sd : storage.dirIterable(null)) {
+      File curDir = sd.getCurrentDir();
+      LOG.info("In directory {}", curDir);
+      File[] files = curDir.listFiles();
+      Arrays.sort(files);
+      for (File f : files) {
+        LOG.info("  file {}; len = {}",  f.getAbsolutePath(), f.length());
+      }
+    }
+  }
   
   /** get the fsImage*/
   public static FSImage getFSImage(NameNode node) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 2e49674..d0c0a63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -45,6 +45,8 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.cli.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -110,7 +112,7 @@ public class TestCheckpoint {
     GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
   }
 
-  static final Log LOG = LogFactory.getLog(TestCheckpoint.class); 
+  static final Logger LOG = LoggerFactory.getLogger(TestCheckpoint.class);
   static final String NN_METRICS = "NameNodeActivity";
   
   static final long seed = 0xDEADBEEFL;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
index d64004f..d42f43c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
@@ -28,8 +28,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -74,7 +74,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LEASE_RECHECK_IN
  */
 public class TestDeleteRace {
   private static final int BLOCK_SIZE = 4096;
-  private static final Log LOG = LogFactory.getLog(TestDeleteRace.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestDeleteRace.class);
   private static final Configuration conf = new HdfsConfiguration();
   private MiniDFSCluster cluster;
 
@@ -171,7 +171,7 @@ public class TestDeleteRace {
         inodeMap.put(fileINode);
         LOG.info("Deleted" + path);
       } catch (Exception e) {
-        LOG.info(e);
+        LOG.info(e.toString());
       }
     }
   }
@@ -196,7 +196,7 @@ public class TestDeleteRace {
         fs.rename(from, to);
         LOG.info("Renamed " + from + " to " + to);
       } catch (Exception e) {
-        LOG.info(e);
+        LOG.info(e.toString());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 7477824..9031b2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -35,8 +35,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -82,7 +82,7 @@ public class TestSaveNamespace {
     GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
   }
   
-  private static final Log LOG = LogFactory.getLog(TestSaveNamespace.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestSaveNamespace.class);
 
   private static class FaultySaveImage implements Answer<Void> {
     private int count = 0;
@@ -325,7 +325,7 @@ public class TestSaveNamespace {
         try {
           fsn.close();
         } catch (Throwable t) {
-          LOG.fatal("Failed to shut down", t);
+          LOG.error("Failed to shut down", t);
         }
       }
     }
@@ -625,7 +625,7 @@ public class TestSaveNamespace {
       cluster.getNameNodeRpc().saveNamespace(0, 0);
       fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
     } finally {
-      IOUtils.cleanup(LOG, out, fs);
+      IOUtils.cleanupWithLogger(LOG, out, fs);
       cluster.shutdown();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
index c1521bd..1166dae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
@@ -29,8 +29,8 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -69,7 +69,7 @@ import org.mockito.invocation.InvocationOnMock;
 
 public class TestDNFencing {
   
-  protected static final Log LOG = LogFactory.getLog(TestDNFencing.class);
+  protected static final Logger LOG = LoggerFactory.getLogger(TestDNFencing.class);
   private static final String TEST_FILE = "/testStandbyIsHot";
   private static final Path TEST_FILE_PATH = new Path(TEST_FILE);
   private static final int SMALL_BLOCK = 1024;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index f9e1fd0..e1f00f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -28,8 +28,8 @@ import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -70,12 +70,12 @@ import com.google.common.base.Supplier;
  */
 public class TestPipelinesFailover {
   static {
-    GenericTestUtils.setLogLevel(LogFactory.getLog(RetryInvocationHandler
-            .class), Level.ALL);
+    GenericTestUtils.setLogLevel(LoggerFactory.getLogger(RetryInvocationHandler
+            .class), org.slf4j.event.Level.DEBUG);
     DFSTestUtil.setNameNodeLogLevel(Level.ALL);
   }
   
-  protected static final Log LOG = LogFactory.getLog(
+  protected static final Logger LOG = LoggerFactory.getLogger(
       TestPipelinesFailover.class);
   private static final Path TEST_PATH =
     new Path("/test-file");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79c97f6a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 4849b54..b9880f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -21,8 +21,8 @@ import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -72,7 +72,7 @@ public class TestStandbyCheckpoints {
   private final Random random = new Random();
   protected File tmpOivImgDir;
   
-  private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestStandbyCheckpoints.class);
 
   @SuppressWarnings("rawtypes")
   @Before


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


[35/50] [abbrv] hadoop git commit: YARN-8015. Support all types of placement constraint support for Capacity Scheduler. Contributed by Weiwei Yang.

Posted by sh...@apache.org.
YARN-8015. Support all types of placement constraint support for Capacity Scheduler. Contributed by Weiwei Yang.


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

Branch: refs/heads/HDFS-12943
Commit: 1ac01444a24faee6f74f2e83d9521eb4e0be651b
Parents: b021249
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 23 10:05:43 2018 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 23 10:05:43 2018 +0530

----------------------------------------------------------------------
 .../SingleConstraintAppPlacementAllocator.java  | 175 ++------
 .../yarn/server/resourcemanager/MockRM.java     |  35 +-
 ...estSchedulingRequestContainerAllocation.java | 438 ++++++++++++++++++-
 ...stSingleConstraintAppPlacementAllocator.java |  78 ----
 4 files changed, 509 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
index 914f35d..54e4666 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
@@ -19,18 +19,15 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableSet;
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
-import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -48,12 +45,12 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE;
 import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
 
 /**
@@ -70,7 +67,6 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
 
   private SchedulingRequest schedulingRequest = null;
   private String targetNodePartition;
-  private Set<String> targetAllocationTags;
   private AllocationTagsManager allocationTagsManager;
   private PlacementConstraintManager placementConstraintManager;
 
@@ -239,135 +235,55 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
           "Only GUARANTEED execution type is supported.");
     }
 
-    // Node partition
-    String nodePartition = null;
-    // Target allocation tags
-    Set<String> targetAllocationTags = null;
-
-    PlacementConstraint constraint =
-        newSchedulingRequest.getPlacementConstraint();
-
-    if (constraint != null) {
-      // We only accept SingleConstraint
-      PlacementConstraint.AbstractConstraint ac = constraint
-          .getConstraintExpr();
-      if (!(ac instanceof PlacementConstraint.SingleConstraint)) {
-        throwExceptionWithMetaInfo("Only accepts "
-            + PlacementConstraint.SingleConstraint.class.getName()
-                + " as constraint-expression. Rejecting the new added "
-            + "constraint-expression.class=" + ac.getClass().getName());
-      }
-
-      PlacementConstraint.SingleConstraint singleConstraint =
-          (PlacementConstraint.SingleConstraint) ac;
-
-      // Make sure it is an anti-affinity request (actually this implementation
-      // should be able to support both affinity / anti-affinity without much
-      // effort. Considering potential test effort required. Limit to
-      // anti-affinity to intra-app and scope is node.
-      if (!singleConstraint.getScope().equals(PlacementConstraints.NODE)) {
-        throwExceptionWithMetaInfo(
-            "Only support scope=" + PlacementConstraints.NODE
-                + "now. PlacementConstraint=" + singleConstraint);
-      }
-
-      if (singleConstraint.getMinCardinality() != 0
-          || singleConstraint.getMaxCardinality() != 0) {
-        throwExceptionWithMetaInfo(
-            "Only support anti-affinity, which is: minCardinality=0, "
-                + "maxCardinality=1");
-      }
-
-      Set<PlacementConstraint.TargetExpression> targetExpressionSet =
-          singleConstraint.getTargetExpressions();
-      if (targetExpressionSet == null || targetExpressionSet.isEmpty()) {
-        throwExceptionWithMetaInfo(
-            "TargetExpression should not be null or empty");
-      }
-
-      for (PlacementConstraint.TargetExpression targetExpression :
-          targetExpressionSet) {
-        // Handle node partition
-        if (targetExpression.getTargetType().equals(
-            PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE)) {
-          // For node attribute target, we only support Partition now. And once
-          // YARN-3409 is merged, we will support node attribute.
-          if (!targetExpression.getTargetKey().equals(NODE_PARTITION)) {
-            throwExceptionWithMetaInfo("When TargetType="
-                + PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE
-                + " only " + NODE_PARTITION + " is accepted as TargetKey.");
-          }
-
-          if (nodePartition != null) {
-            // This means we have duplicated node partition entry
-            // inside placement constraint, which might be set by mistake.
-            throwExceptionWithMetaInfo(
-                "Only one node partition targetExpression is allowed");
-          }
-
-          Set<String> values = targetExpression.getTargetValues();
-          if (values == null || values.isEmpty()) {
-            nodePartition = RMNodeLabelsManager.NO_LABEL;
-            continue;
-          }
+    this.targetNodePartition = validateAndGetTargetNodePartition(
+        newSchedulingRequest.getPlacementConstraint());
+    this.schedulingRequest = new SchedulingRequestPBImpl(
+        ((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
 
-          if (values.size() > 1) {
-            throwExceptionWithMetaInfo("Inside one targetExpression, we only "
-                + "support affinity to at most one node partition now");
-          }
 
-          nodePartition = values.iterator().next();
-        } else if (targetExpression.getTargetType().equals(
-            PlacementConstraint.TargetExpression.TargetType.ALLOCATION_TAG)) {
-          // Handle allocation tags
-          if (targetAllocationTags != null) {
-            // This means we have duplicated AllocationTag expressions entries
-            // inside placement constraint, which might be set by mistake.
-            throwExceptionWithMetaInfo(
-                "Only one AllocationTag targetExpression is allowed");
-          }
+    LOG.info("Successfully added SchedulingRequest to app="
+        + appSchedulingInfo.getApplicationAttemptId()
+        + " placementConstraint=["
+        + schedulingRequest.getPlacementConstraint()
+        + "]. nodePartition=" + targetNodePartition);
+  }
 
-          if (targetExpression.getTargetValues() == null ||
-              targetExpression.getTargetValues().isEmpty()) {
-            throwExceptionWithMetaInfo("Failed to find allocation tags from "
-                + "TargetExpressions or couldn't find self-app target.");
+  // Tentatively find out potential exist node-partition in the placement
+  // constraint and set as the app's primary node-partition.
+  // Currently only single constraint is handled.
+  private String validateAndGetTargetNodePartition(
+      PlacementConstraint placementConstraint) {
+    String nodePartition = RMNodeLabelsManager.NO_LABEL;
+    if (placementConstraint != null &&
+        placementConstraint.getConstraintExpr() != null) {
+      PlacementConstraint.AbstractConstraint ac =
+          placementConstraint.getConstraintExpr();
+      if (ac != null && ac instanceof PlacementConstraint.SingleConstraint) {
+        PlacementConstraint.SingleConstraint singleConstraint =
+            (PlacementConstraint.SingleConstraint) ac;
+        for (PlacementConstraint.TargetExpression targetExpression :
+            singleConstraint.getTargetExpressions()) {
+          // Handle node partition
+          if (targetExpression.getTargetType().equals(NODE_ATTRIBUTE) &&
+              targetExpression.getTargetKey().equals(NODE_PARTITION)) {
+            Set<String> values = targetExpression.getTargetValues();
+            if (values == null || values.isEmpty()) {
+              continue;
+            }
+            if (values.size() > 1) {
+              throwExceptionWithMetaInfo(
+                  "Inside one targetExpression, we only support"
+                      + " affinity to at most one node partition now");
+            }
+            nodePartition = values.iterator().next();
+            if (nodePartition != null) {
+              break;
+            }
           }
-
-          targetAllocationTags = new HashSet<>(
-              targetExpression.getTargetValues());
         }
       }
-
-      if (targetAllocationTags == null) {
-        // That means we don't have ALLOCATION_TAG specified
-        throwExceptionWithMetaInfo(
-            "Couldn't find target expression with type == ALLOCATION_TAG,"
-                + " it is required to include one and only one target"
-                + " expression with type == ALLOCATION_TAG");
-      }
-    }
-
-    // If this scheduling request doesn't contain a placement constraint,
-    // we set allocation tags an empty set.
-    if (targetAllocationTags == null) {
-      targetAllocationTags = ImmutableSet.of();
-    }
-
-    if (nodePartition == null) {
-      nodePartition = RMNodeLabelsManager.NO_LABEL;
     }
-
-    // Validation is done. set local results:
-    this.targetNodePartition = nodePartition;
-    this.targetAllocationTags = targetAllocationTags;
-
-    this.schedulingRequest = new SchedulingRequestPBImpl(
-        ((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
-
-    LOG.info("Successfully added SchedulingRequest to app=" + appSchedulingInfo
-        .getApplicationAttemptId() + " targetAllocationTags=[" + StringUtils
-        .join(",", targetAllocationTags) + "]. nodePartition="
-        + targetNodePartition);
+    return nodePartition;
   }
 
   @Override
@@ -515,11 +431,6 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
     return targetNodePartition;
   }
 
-  @VisibleForTesting
-  Set<String> getTargetAllocationTags() {
-    return targetAllocationTags;
-  }
-
   @Override
   public void initialize(AppSchedulingInfo appSchedulingInfo,
       SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index eb4c626..2ad4391 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -513,6 +513,19 @@ public class MockRM extends ResourceManager {
     return submitApp(masterMemory, false);
   }
 
+  public RMApp submitApp(int masterMemory, Set<String> appTags)
+      throws Exception {
+    Resource resource = Resource.newInstance(masterMemory, 0);
+    ResourceRequest amResourceRequest = ResourceRequest.newInstance(
+        Priority.newInstance(0), ResourceRequest.ANY, resource, 1);
+    return submitApp(Collections.singletonList(amResourceRequest), "",
+        UserGroupInformation.getCurrentUser().getShortUserName(), null, false,
+        null, super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true,
+        false, false, null, 0, null, true, Priority.newInstance(0), null,
+        null, null, appTags);
+  }
+
   public RMApp submitApp(int masterMemory, Priority priority) throws Exception {
     Resource resource = Resource.newInstance(masterMemory, 0);
     return submitApp(resource, "", UserGroupInformation.getCurrentUser()
@@ -732,8 +745,23 @@ public class MockRM extends ResourceManager {
       LogAggregationContext logAggregationContext,
       boolean cancelTokensWhenComplete, Priority priority, String amLabel,
       Map<ApplicationTimeoutType, Long> applicationTimeouts,
-      ByteBuffer tokensConf)
-      throws Exception {
+      ByteBuffer tokensConf) throws Exception {
+    return submitApp(amResourceRequests, name, user, acls, unmanaged, queue,
+        maxAppAttempts, ts, appType, waitForAccepted, keepContainers,
+        isAppIdProvided, applicationId, attemptFailuresValidityInterval,
+        logAggregationContext, cancelTokensWhenComplete, priority, amLabel,
+        applicationTimeouts, tokensConf, null);
+  }
+
+  public RMApp submitApp(List<ResourceRequest> amResourceRequests, String name,
+      String user, Map<ApplicationAccessType, String> acls, boolean unmanaged,
+      String queue, int maxAppAttempts, Credentials ts, String appType,
+      boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided,
+      ApplicationId applicationId, long attemptFailuresValidityInterval,
+      LogAggregationContext logAggregationContext,
+      boolean cancelTokensWhenComplete, Priority priority, String amLabel,
+      Map<ApplicationTimeoutType, Long> applicationTimeouts,
+      ByteBuffer tokensConf, Set<String> applicationTags) throws Exception {
     ApplicationId appId = isAppIdProvided ? applicationId : null;
     ApplicationClientProtocol client = getClientRMService();
     if (! isAppIdProvided) {
@@ -749,6 +777,9 @@ public class MockRM extends ResourceManager {
     sub.setApplicationId(appId);
     sub.setApplicationName(name);
     sub.setMaxAppAttempts(maxAppAttempts);
+    if (applicationTags != null) {
+      sub.setApplicationTags(applicationTags);
+    }
     if (applicationTimeouts != null && applicationTimeouts.size() > 0) {
       sub.setApplicationTimeouts(applicationTimeouts);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java
index f23fd8f..26c709f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestSchedulingRequestContainerAllocation.java
@@ -26,12 +26,18 @@ import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
-import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.TargetApplicationsNamespace;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceSizing;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.AllocationTagNamespaceType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
@@ -46,10 +52,24 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTag;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.PlacementTargets.allocationTagWithNamespace;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.and;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.cardinality;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetIn;
+import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.targetNotIn;
 
 public class TestSchedulingRequestContainerAllocation {
-  private final int GB = 1024;
+  private static final int GB = 1024;
 
   private YarnConfiguration conf;
 
@@ -435,8 +455,7 @@ public class TestSchedulingRequestContainerAllocation {
 
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
 
-    PlacementConstraint constraint = PlacementConstraints
-        .targetNotIn("node", allocationTag("t1"))
+    PlacementConstraint constraint = targetNotIn("node", allocationTag("t1"))
         .build();
     SchedulingRequest sc = SchedulingRequest
         .newInstance(0, Priority.newInstance(1),
@@ -477,4 +496,413 @@ public class TestSchedulingRequestContainerAllocation {
 
     rm1.close();
   }
+
+  private void doNodeHeartbeat(MockNM... nms) throws Exception {
+    for (MockNM nm : nms) {
+      nm.nodeHeartbeat(true);
+    }
+  }
+
+  private List<Container> waitForAllocation(int allocNum, int timeout,
+      MockAM am, MockNM... nms) throws Exception {
+    final List<Container> result = new ArrayList<>();
+    GenericTestUtils.waitFor(() -> {
+      try {
+        AllocateResponse response = am.schedule();
+        List<Container> allocated = response.getAllocatedContainers();
+        System.out.println("Expecting allocation: " + allocNum
+            + ", actual allocation: " + allocated.size());
+        for (Container c : allocated) {
+          System.out.println("Container " + c.getId().toString()
+              + " is allocated on node: " + c.getNodeId().toString()
+              + ", allocation tags: "
+              + String.join(",", c.getAllocationTags()));
+        }
+        result.addAll(allocated);
+        if (result.size() == allocNum) {
+          return true;
+        }
+        doNodeHeartbeat(nms);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+      return false;
+    }, 500, timeout);
+    return result;
+  }
+
+  private static SchedulingRequest schedulingRequest(int requestId,
+      int containers, int cores, int mem, PlacementConstraint constraint,
+      String... tags) {
+    return schedulingRequest(1, requestId, containers, cores, mem,
+        ExecutionType.GUARANTEED, constraint, tags);
+  }
+
+  private static SchedulingRequest schedulingRequest(
+      int priority, long allocReqId, int containers, int cores, int mem,
+      ExecutionType execType, PlacementConstraint constraint, String... tags) {
+    return SchedulingRequest.newBuilder()
+        .priority(Priority.newInstance(priority))
+        .allocationRequestId(allocReqId)
+        .allocationTags(new HashSet<>(Arrays.asList(tags)))
+        .executionType(ExecutionTypeRequest.newInstance(execType, true))
+        .resourceSizing(
+            ResourceSizing.newInstance(containers,
+                Resource.newInstance(mem, cores)))
+        .placementConstraintExpression(constraint)
+        .build();
+  }
+
+  private int getContainerNodesNum(List<Container> containers) {
+    Set<NodeId> nodes = new HashSet<>();
+    if (containers != null) {
+      containers.forEach(c -> nodes.add(c.getNodeId()));
+    }
+    return nodes.size();
+  }
+
+  @Test(timeout = 30000L)
+  public void testInterAppCompositeConstraints() throws Exception {
+    // This test both intra and inter app constraints.
+    // Including simple affinity, anti-affinity, cardinality constraints,
+    // and simple AND composite constraints.
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234", 100*GB, 100);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 100*GB, 100);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 100*GB, 100);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 100*GB, 100);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 100*GB, 100);
+
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("hbase"));
+      MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
+
+      // App1 (hbase)
+      // h1: hbase-master(1)
+      // h2: hbase-master(1)
+      // h3:
+      // h4:
+      // h5:
+      PlacementConstraint pc = targetNotIn("node",
+          allocationTag("hbase-master")).build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 2048, pc, "hbase-master")));
+      List<Container> allocated = waitForAllocation(2, 3000, am1, nm1, nm2);
+
+      // 2 containers allocated
+      Assert.assertEquals(2, allocated.size());
+      // containers should be distributed on 2 different nodes
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      // App1 (hbase)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1)
+      // h4: hbase-rs(1)
+      // h5:
+      pc = targetNotIn("node", allocationTag("hbase-rs")).build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(2, 4, 1, 1024, pc, "hbase-rs")));
+      allocated = waitForAllocation(4, 3000, am1, nm1, nm2, nm3, nm4, nm5);
+
+      Assert.assertEquals(4, allocated.size());
+      Assert.assertEquals(4, getContainerNodesNum(allocated));
+
+      // App2 (web-server)
+      // Web server instance has 2 instance and non of them can be co-allocated
+      // with hbase-master.
+      RMApp app2 = rm.submitApp(1*GB, ImmutableSet.of("web-server"));
+      MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+
+      // App2 (web-server)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1), ws-inst(1)
+      // h4: hbase-rs(1), ws-inst(1)
+      // h5:
+      pc = and(
+          targetIn("node", allocationTagWithNamespace(
+              new TargetApplicationsNamespace.All().toString(),
+              "hbase-master")),
+          targetNotIn("node", allocationTag("ws-inst"))).build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 2048, pc, "ws-inst")));
+      allocated = waitForAllocation(2, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(2, allocated.size());
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("If ws-inst is allocated to a node,"
+                + " this node should have inherited the ws-inst tag ",
+            rmNode.getAllocationTagsWithCount().get("ws-inst") == 1);
+        Assert.assertTrue("ws-inst should be co-allocated to "
+                + "hbase-master nodes",
+            rmNode.getAllocationTagsWithCount().get("hbase-master") == 1);
+      }
+
+      // App3 (ws-servant)
+      // App3 has multiple instances that must be co-allocated
+      // with app2 server instance, and each node cannot have more than
+      // 3 instances.
+      RMApp app3 = rm.submitApp(1*GB, ImmutableSet.of("ws-servants"));
+      MockAM am3 = MockRM.launchAndRegisterAM(app3, rm, nm3);
+
+
+      // App3 (ws-servant)
+      // h1: hbase-rs(1), hbase-master(1)
+      // h2: hbase-rs(1), hbase-master(1)
+      // h3: hbase-rs(1), ws-inst(1), ws-servant(3)
+      // h4: hbase-rs(1), ws-inst(1), ws-servant(3)
+      // h5:
+      pc = and(
+          targetIn("node", allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppTag("web-server").toString(),
+              "ws-inst")),
+          cardinality("node", 0, 2, "ws-servant")).build();
+      am3.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 10, 1, 512, pc, "ws-servant")));
+      // total 6 containers can be allocated due to cardinality constraint
+      // each round, 2 containers can be allocated
+      allocated = waitForAllocation(6, 10000, am3, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(6, allocated.size());
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("Node has ws-servant allocated must have 3 instances",
+            rmNode.getAllocationTagsWithCount().get("ws-servant") == 3);
+        Assert.assertTrue("Every ws-servant container should be co-allocated"
+                + " with ws-inst",
+            rmNode.getAllocationTagsWithCount().get("ws-inst") == 1);
+      }
+    } finally {
+      rm.stop();
+    }
+  }
+
+  @Test(timeout = 30000L)
+  public void testMultiAllocationTagsConstraints() throws Exception {
+    // This test simulates to use PC to avoid port conflicts
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234", 10*GB, 10);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 10*GB, 10);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 10*GB, 10);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 10*GB, 10);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 10*GB, 10);
+
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("server1"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm1);
+      RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+      MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+      am1.registerAppAttempt();
+
+      // App1 uses ports: 7000, 8000 and 9000
+      String[] server1Ports =
+          new String[] {"port_6000", "port_7000", "port_8000"};
+      PlacementConstraint pc = targetNotIn("node",
+          allocationTagWithNamespace(AllocationTagNamespaceType.ALL.toString(),
+              server1Ports))
+          .build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 1024, pc, server1Ports)));
+      List<Container> allocated = waitForAllocation(2, 3000,
+          am1, nm1, nm2, nm3, nm4, nm5);
+
+      // 2 containers allocated
+      Assert.assertEquals(2, allocated.size());
+      // containers should be distributed on 2 different nodes
+      Assert.assertEquals(2, getContainerNodesNum(allocated));
+
+      // App1 uses ports: 6000
+      String[] server2Ports = new String[] {"port_6000"};
+      RMApp app2 = rm.submitApp(1*GB, ImmutableSet.of("server2"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm2);
+      RMAppAttempt app2attempt1 = app2.getCurrentAppAttempt();
+      MockAM am2 = rm.sendAMLaunched(app2attempt1.getAppAttemptId());
+      am2.registerAppAttempt();
+
+      pc = targetNotIn("node",
+          allocationTagWithNamespace(AllocationTagNamespaceType.ALL.toString(),
+              server2Ports))
+          .build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 3, 1, 1024, pc, server2Ports)));
+      allocated = waitForAllocation(3, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(3, allocated.size());
+      Assert.assertEquals(3, getContainerNodesNum(allocated));
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("server2 should not co-allocate to server1 as"
+                + " they both need to use port 6000",
+            rmNode.getAllocationTagsWithCount().get("port_6000") == 1);
+        Assert.assertFalse(rmNode.getAllocationTagsWithCount()
+            .containsKey("port_7000"));
+        Assert.assertFalse(rmNode.getAllocationTagsWithCount()
+            .containsKey("port_8000"));
+      }
+    } finally {
+      rm.stop();
+    }
+  }
+
+  @Test(timeout = 30000L)
+  public void testInterAppConstraintsWithNamespaces() throws Exception {
+    // This test verifies inter-app constraints with namespaces
+    // not-self/app-id/app-tag
+    YarnConfiguration config = new YarnConfiguration();
+    config.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
+        YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
+    config.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+    MockRM rm = new MockRM(config);
+    try {
+      rm.start();
+
+      MockNM nm1 = rm.registerNode("192.168.0.1:1234:", 100*GB, 100);
+      MockNM nm2 = rm.registerNode("192.168.0.2:1234", 100*GB, 100);
+      MockNM nm3 = rm.registerNode("192.168.0.3:1234", 100*GB, 100);
+      MockNM nm4 = rm.registerNode("192.168.0.4:1234", 100*GB, 100);
+      MockNM nm5 = rm.registerNode("192.168.0.5:1234", 100*GB, 100);
+
+      ApplicationId app5Id = null;
+      Map<ApplicationId, List<Container>> allocMap = new HashMap<>();
+      // 10 apps and all containers are attached with foo tag
+      for (int i = 0; i<10; i++) {
+        // App1 ~ app5 tag "former5"
+        // App6 ~ app10 tag "latter5"
+        String applicationTag = i<5 ? "former5" : "latter5";
+        RMApp app = rm.submitApp(1*GB, ImmutableSet.of(applicationTag));
+        // Allocate AM container on nm1
+        doNodeHeartbeat(nm1, nm2, nm3, nm4, nm5);
+        RMAppAttempt attempt = app.getCurrentAppAttempt();
+        MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
+        am.registerAppAttempt();
+
+        PlacementConstraint pc = targetNotIn("node", allocationTag("foo"))
+            .build();
+        am.addSchedulingRequest(
+            ImmutableList.of(
+                schedulingRequest(1, 3, 1, 1024, pc, "foo")));
+        List<Container> allocated = waitForAllocation(3, 3000,
+            am, nm1, nm2, nm3, nm4, nm5);
+        // Memorize containers that has app5 foo
+        if (i == 5) {
+          app5Id = am.getApplicationAttemptId().getApplicationId();
+        }
+        allocMap.put(am.getApplicationAttemptId().getApplicationId(),
+            allocated);
+      }
+
+      Assert.assertNotNull(app5Id);
+      Assert.assertEquals(3, getContainerNodesNum(allocMap.get(app5Id)));
+
+      // *** app-id
+      // Submit another app, use app-id constraint against app5
+      RMApp app1 = rm.submitApp(1*GB, ImmutableSet.of("xyz"));
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm1);
+      RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
+      MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
+      am1.registerAppAttempt();
+
+      PlacementConstraint pc = targetIn("node",
+          allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppID(app5Id).toString(),
+              "foo"))
+          .build();
+      am1.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 3, 1, 1024, pc, "foo")));
+      List<Container> allocated = waitForAllocation(3, 3000,
+          am1, nm1, nm2, nm3, nm4, nm5);
+
+      ConcurrentMap<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+      List<Container> app5Alloc = allocMap.get(app5Id);
+      for (Container c : allocated) {
+        RMNode rmNode = rmNodes.get(c.getNodeId());
+        Assert.assertNotNull(rmNode);
+        Assert.assertTrue("This app is affinity with app-id/app5/foo "
+                + "containers",
+            app5Alloc.stream().anyMatch(
+                c5 -> c5.getNodeId() == c.getNodeId()));
+      }
+
+      // *** app-tag
+      RMApp app2 = rm.submitApp(1*GB);
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm2);
+      RMAppAttempt app2attempt1 = app2.getCurrentAppAttempt();
+      MockAM am2 = rm.sendAMLaunched(app2attempt1.getAppAttemptId());
+      am2.registerAppAttempt();
+
+      pc = targetNotIn("node",
+          allocationTagWithNamespace(
+              new TargetApplicationsNamespace.AppTag("xyz").toString(),
+              "foo"))
+          .build();
+      am2.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 2, 1, 1024, pc, "foo")));
+      allocated = waitForAllocation(2, 3000, am2, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(2, allocated.size());
+
+      // none of them can be allocated to nodes that has app5 foo containers
+      for (Container c : app5Alloc) {
+        Assert.assertNotEquals(c.getNodeId(),
+            allocated.iterator().next().getNodeId());
+      }
+
+      // *** not-self
+      RMApp app3 = rm.submitApp(1*GB);
+      // Allocate AM container on nm1
+      doNodeHeartbeat(nm3);
+      RMAppAttempt app3attempt1 = app3.getCurrentAppAttempt();
+      MockAM am3 = rm.sendAMLaunched(app3attempt1.getAppAttemptId());
+      am3.registerAppAttempt();
+
+      pc = cardinality("node",
+          new TargetApplicationsNamespace.NotSelf().toString(),
+          1, 1, "foo").build();
+      am3.addSchedulingRequest(
+          ImmutableList.of(
+              schedulingRequest(1, 1, 1, 1024, pc, "foo")));
+      allocated = waitForAllocation(1, 3000, am3, nm1, nm2, nm3, nm4, nm5);
+      Assert.assertEquals(1, allocated.size());
+      // All 5 containers should be allocated
+      Assert.assertTrue(rmNodes.get(allocated.iterator().next().getNodeId())
+          .getAllocationTagsWithCount().get("foo") == 2);
+    } finally {
+      rm.stop();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ac01444/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
index ccf4281..902c6d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/TestSingleConstraintAppPlacementAllocator.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
 
-import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTags;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
@@ -131,8 +130,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Valid (with partition)
@@ -147,8 +144,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .build()).resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("x", allocator.getTargetNodePartition());
 
     // Valid (without specifying node partition)
@@ -162,8 +157,6 @@ public class TestSingleConstraintAppPlacementAllocator {
         .resourceSizing(
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Valid (with application Id target)
@@ -178,8 +171,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
         .build());
     // Allocation tags should not include application Id
-    Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
-        allocator.getTargetAllocationTags());
     Assert.assertEquals("", allocator.getTargetNodePartition());
 
     // Invalid (without sizing)
@@ -200,75 +191,6 @@ public class TestSingleConstraintAppPlacementAllocator {
             .targetNotIn(PlacementConstraints.NODE).build())
         .build(), true);
 
-    // Invalid (with multiple allocation tags expression specified)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.NODE,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("reducer"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (with multiple node partition target expression specified)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.NODE,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets
-                    .allocationTag(""),
-                PlacementConstraints.PlacementTargets.nodePartition("x"))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not anti-affinity cardinality)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetCardinality(PlacementConstraints.NODE, 1, 2,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not anti-affinity cardinality)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetCardinality(PlacementConstraints.NODE, 0, 2,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
-    // Invalid (not NODE scope)
-    assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
-        ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
-        .allocationRequestId(10L).priority(Priority.newInstance(1))
-        .placementConstraintExpression(PlacementConstraints
-            .targetNotIn(PlacementConstraints.RACK,
-                PlacementConstraints.PlacementTargets
-                    .allocationTag("mapper", "reducer"),
-                PlacementConstraints.PlacementTargets.nodePartition(""))
-            .build()).resourceSizing(
-            ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
-        .build(), true);
-
     // Invalid (not GUARANTEED)
     assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
         ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC))


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


[18/50] [abbrv] hadoop git commit: HDFS-13750. RBF: Router ID in RouterRpcClient is always null. Contributed by Takanobu Asanuma.

Posted by sh...@apache.org.
HDFS-13750. RBF: Router ID in RouterRpcClient is always null. Contributed by Takanobu Asanuma.


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

Branch: refs/heads/HDFS-12943
Commit: 01ff8178148790f7b0112058cf08d23d031b6868
Parents: e3d73bb
Author: Yiqun Lin <yq...@apache.org>
Authored: Mon Aug 20 23:01:59 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Mon Aug 20 23:01:59 2018 +0800

----------------------------------------------------------------------
 .../server/federation/router/RouterRpcClient.java | 18 ++++++++++--------
 .../server/federation/router/RouterRpcServer.java |  3 ++-
 .../hdfs/server/federation/MockResolver.java      |  3 +++
 .../hdfs/server/federation/router/TestRouter.java | 18 ++++++++++++++++++
 4 files changed, 33 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01ff8178/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
index 3eb7241..56ca55f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -92,8 +92,8 @@ public class RouterRpcClient {
       LoggerFactory.getLogger(RouterRpcClient.class);
 
 
-  /** Router identifier. */
-  private final String routerId;
+  /** Router using this RPC client. */
+  private final Router router;
 
   /** Interface to identify the active NN for a nameservice or blockpool ID. */
   private final ActiveNamenodeResolver namenodeResolver;
@@ -116,12 +116,13 @@ public class RouterRpcClient {
    * Create a router RPC client to manage remote procedure calls to NNs.
    *
    * @param conf Hdfs Configuation.
+   * @param router A router using this RPC client.
    * @param resolver A NN resolver to determine the currently active NN in HA.
    * @param monitor Optional performance monitor.
    */
-  public RouterRpcClient(Configuration conf, String identifier,
+  public RouterRpcClient(Configuration conf, Router router,
       ActiveNamenodeResolver resolver, RouterRpcMonitor monitor) {
-    this.routerId = identifier;
+    this.router = router;
 
     this.namenodeResolver = resolver;
 
@@ -343,7 +344,8 @@ public class RouterRpcClient {
 
     if (namenodes == null || namenodes.isEmpty()) {
       throw new IOException("No namenodes to invoke " + method.getName() +
-          " with params " + Arrays.toString(params) + " from " + this.routerId);
+          " with params " + Arrays.toString(params) + " from "
+          + router.getRouterId());
     }
 
     Object ret = null;
@@ -1126,7 +1128,7 @@ public class RouterRpcClient {
       String msg = "Not enough client threads " + active + "/" + total;
       LOG.error(msg);
       throw new StandbyException(
-          "Router " + routerId + " is overloaded: " + msg);
+          "Router " + router.getRouterId() + " is overloaded: " + msg);
     } catch (InterruptedException ex) {
       LOG.error("Unexpected error while invoking API: {}", ex.getMessage());
       throw new IOException(
@@ -1150,7 +1152,7 @@ public class RouterRpcClient {
 
     if (namenodes == null || namenodes.isEmpty()) {
       throw new IOException("Cannot locate a registered namenode for " + nsId +
-          " from " + this.routerId);
+          " from " + router.getRouterId());
     }
     return namenodes;
   }
@@ -1171,7 +1173,7 @@ public class RouterRpcClient {
 
     if (namenodes == null || namenodes.isEmpty()) {
       throw new IOException("Cannot locate a registered namenode for " + bpId +
-          " from " + this.routerId);
+          " from " + router.getRouterId());
     }
     return namenodes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01ff8178/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index fe54993..2deda9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -196,6 +196,7 @@ public class RouterRpcServer extends AbstractService
    * Construct a router RPC server.
    *
    * @param configuration HDFS Configuration.
+   * @param router A router using this RPC server.
    * @param nnResolver The NN resolver instance to determine active NNs in HA.
    * @param fileResolver File resolver to resolve file paths to subclusters.
    * @throws IOException If the RPC server could not be created.
@@ -291,7 +292,7 @@ public class RouterRpcServer extends AbstractService
     this.rpcMonitor = ReflectionUtils.newInstance(rpcMonitorClass, conf);
 
     // Create the client
-    this.rpcClient = new RouterRpcClient(this.conf, this.router.getRouterId(),
+    this.rpcClient = new RouterRpcClient(this.conf, this.router,
         this.namenodeResolver, this.rpcMonitor);
 
     // Initialize modules

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01ff8178/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
index 36cce39..f5636ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -129,6 +129,9 @@ public class MockResolver
     // Return a copy of the list because it is updated periodically
     List<? extends FederationNamenodeContext> namenodes =
         this.resolver.get(nameserviceId);
+    if (namenodes == null) {
+      namenodes = new ArrayList<>();
+    }
     return Collections.unmodifiableList(new ArrayList<>(namenodes));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01ff8178/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
index f8cf009..db4be29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -185,4 +187,20 @@ public class TestRouter {
     router.stop();
     router.close();
   }
+
+  @Test
+  public void testRouterIDInRouterRpcClient() throws Exception {
+
+    Router router = new Router();
+    router.init(new RouterConfigBuilder(conf).rpc().build());
+    router.setRouterId("Router-0");
+    RemoteMethod remoteMethod = mock(RemoteMethod.class);
+
+    intercept(IOException.class, "Router-0",
+        () -> router.getRpcServer().getRPCClient()
+            .invokeSingle("ns0", remoteMethod));
+
+    router.stop();
+    router.close();
+  }
 }


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


[11/50] [abbrv] hadoop git commit: HADOOP-9214. Create a new touch command to allow modifying atime and mtime. Contributed by Hrishikesh Gadre.

Posted by sh...@apache.org.
HADOOP-9214. Create a new touch command to allow modifying atime and mtime. Contributed by Hrishikesh Gadre.


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

Branch: refs/heads/HDFS-12943
Commit: 60ffec9f7921a50aff20434c1042b16fa59240f7
Parents: a17eed1
Author: Xiao Chen <xi...@apache.org>
Authored: Fri Aug 17 10:53:22 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Fri Aug 17 11:18:09 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/shell/FsCommand.java   |   2 +-
 .../java/org/apache/hadoop/fs/shell/Touch.java  |  85 --------
 .../apache/hadoop/fs/shell/TouchCommands.java   | 198 +++++++++++++++++++
 .../src/site/markdown/FileSystemShell.md        |  32 +++
 .../org/apache/hadoop/fs/TestFsShellTouch.java  | 103 ++++++++++
 .../src/test/resources/testConf.xml             |  51 +++++
 6 files changed, 385 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
index 4a13414..784bbf3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsCommand.java
@@ -66,7 +66,7 @@ abstract public class FsCommand extends Command {
     factory.registerCommands(Tail.class);
     factory.registerCommands(Head.class);
     factory.registerCommands(Test.class);
-    factory.registerCommands(Touch.class);
+    factory.registerCommands(TouchCommands.class);
     factory.registerCommands(Truncate.class);
     factory.registerCommands(SnapshotCommands.class);
     factory.registerCommands(XAttrCommands.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
deleted file mode 100644
index a6c751e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Touch.java
+++ /dev/null
@@ -1,85 +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.fs.shell;
-
-import java.io.IOException;
-import java.util.LinkedList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.PathIOException;
-import org.apache.hadoop.fs.PathIsDirectoryException;
-import org.apache.hadoop.fs.PathNotFoundException;
-
-/**
- * Unix touch like commands
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-
-class Touch extends FsCommand {
-  public static void registerCommands(CommandFactory factory) {
-    factory.addClass(Touchz.class, "-touchz");
-  }
-
-  /**
-   * (Re)create zero-length file at the specified path.
-   * This will be replaced by a more UNIX-like touch when files may be
-   * modified.
-   */
-  public static class Touchz extends Touch {
-    public static final String NAME = "touchz";
-    public static final String USAGE = "<path> ...";
-    public static final String DESCRIPTION =
-      "Creates a file of zero length " +
-      "at <path> with current time as the timestamp of that <path>. " +
-      "An error is returned if the file exists with non-zero length\n";
-
-    @Override
-    protected void processOptions(LinkedList<String> args) {
-      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE);
-      cf.parse(args);
-    }
-
-    @Override
-    protected void processPath(PathData item) throws IOException {
-      if (item.stat.isDirectory()) {
-        // TODO: handle this
-        throw new PathIsDirectoryException(item.toString());
-      }
-      if (item.stat.getLen() != 0) {
-        throw new PathIOException(item.toString(), "Not a zero-length file");
-      }
-      touchz(item);
-    }
-
-    @Override
-    protected void processNonexistentPath(PathData item) throws IOException {
-      if (!item.parentExists()) {
-        throw new PathNotFoundException(item.toString())
-            .withFullyQualifiedPath(item.path.toUri().toString());
-      }
-      touchz(item);
-    }
-
-    private void touchz(PathData item) throws IOException {
-      item.fs.create(item.path).close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
new file mode 100644
index 0000000..be174b5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java
@@ -0,0 +1,198 @@
+/**
+ * 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.fs.shell;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.PathIsDirectoryException;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Unix touch like commands
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+
+public class TouchCommands extends FsCommand {
+  public static void registerCommands(CommandFactory factory) {
+    factory.addClass(Touchz.class, "-touchz");
+    factory.addClass(Touch.class, "-touch");
+  }
+
+  /**
+   * (Re)create zero-length file at the specified path.
+   * This will be replaced by a more UNIX-like touch when files may be
+   * modified.
+   */
+  public static class Touchz extends TouchCommands {
+    public static final String NAME = "touchz";
+    public static final String USAGE = "<path> ...";
+    public static final String DESCRIPTION =
+      "Creates a file of zero length " +
+      "at <path> with current time as the timestamp of that <path>. " +
+      "An error is returned if the file exists with non-zero length\n";
+
+    @Override
+    protected void processOptions(LinkedList<String> args) {
+      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE);
+      cf.parse(args);
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (item.stat.isDirectory()) {
+        // TODO: handle this
+        throw new PathIsDirectoryException(item.toString());
+      }
+      if (item.stat.getLen() != 0) {
+        throw new PathIOException(item.toString(), "Not a zero-length file");
+      }
+      touchz(item);
+    }
+
+    @Override
+    protected void processNonexistentPath(PathData item) throws IOException {
+      if (!item.parentExists()) {
+        throw new PathNotFoundException(item.toString())
+            .withFullyQualifiedPath(item.path.toUri().toString());
+      }
+      touchz(item);
+    }
+
+    private void touchz(PathData item) throws IOException {
+      item.fs.create(item.path).close();
+    }
+  }
+
+  /**
+   * A UNIX like touch command.
+   */
+  public static class Touch extends TouchCommands {
+    private static final String OPTION_CHANGE_ONLY_MODIFICATION_TIME = "m";
+    private static final String OPTION_CHANGE_ONLY_ACCESS_TIME = "a";
+    private static final String OPTION_USE_TIMESTAMP = "t";
+    private static final String OPTION_DO_NOT_CREATE_FILE = "c";
+
+    public static final String NAME = "touch";
+    public static final String USAGE = "[-" + OPTION_CHANGE_ONLY_ACCESS_TIME
+        + "] [-" + OPTION_CHANGE_ONLY_MODIFICATION_TIME + "] [-"
+        + OPTION_USE_TIMESTAMP + " TIMESTAMP ] [-" + OPTION_DO_NOT_CREATE_FILE
+        + "] <path> ...";
+    public static final String DESCRIPTION =
+        "Updates the access and modification times of the file specified by the"
+            + " <path> to the current time. If the file does not exist, then a zero"
+            + " length file is created at <path> with current time as the timestamp"
+            + " of that <path>.\n"
+            + "-" + OPTION_CHANGE_ONLY_ACCESS_TIME
+            + " Change only the access time \n" + "-"
+            + OPTION_CHANGE_ONLY_MODIFICATION_TIME
+            + " Change only the modification time \n" + "-"
+            + OPTION_USE_TIMESTAMP + " TIMESTAMP"
+            + " Use specified timestamp (in format yyyyMMddHHmmss) instead of current time \n"
+            + "-" + OPTION_DO_NOT_CREATE_FILE + " Do not create any files";
+
+    private boolean changeModTime = false;
+    private boolean changeAccessTime = false;
+    private boolean doNotCreate = false;
+    private String timestamp;
+    private final SimpleDateFormat dateFormat =
+        new SimpleDateFormat("yyyyMMdd:HHmmss");
+
+    @InterfaceAudience.Private
+    @VisibleForTesting
+    public DateFormat getDateFormat() {
+      return dateFormat;
+    }
+
+    @Override
+    protected void processOptions(LinkedList<String> args) {
+      this.timestamp =
+          StringUtils.popOptionWithArgument("-" + OPTION_USE_TIMESTAMP, args);
+
+      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE,
+          OPTION_USE_TIMESTAMP, OPTION_CHANGE_ONLY_ACCESS_TIME,
+          OPTION_CHANGE_ONLY_MODIFICATION_TIME);
+      cf.parse(args);
+      this.changeModTime = cf.getOpt(OPTION_CHANGE_ONLY_MODIFICATION_TIME);
+      this.changeAccessTime = cf.getOpt(OPTION_CHANGE_ONLY_ACCESS_TIME);
+      this.doNotCreate = cf.getOpt(OPTION_DO_NOT_CREATE_FILE);
+    }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (item.stat.isDirectory()) {
+        throw new PathIsDirectoryException(item.toString());
+      }
+      touch(item);
+    }
+
+    @Override
+    protected void processNonexistentPath(PathData item) throws IOException {
+      if (!item.parentExists()) {
+        throw new PathNotFoundException(item.toString())
+            .withFullyQualifiedPath(item.path.toUri().toString());
+      }
+      touch(item);
+    }
+
+    private void touch(PathData item) throws IOException {
+      if (!item.fs.exists(item.path)) {
+        if (doNotCreate) {
+          return;
+        }
+        item.fs.create(item.path).close();
+        if (timestamp != null) {
+          // update the time only if user specified a timestamp using -t option.
+          updateTime(item);
+        }
+      } else {
+        updateTime(item);
+      }
+    }
+
+    private void updateTime(PathData item) throws IOException {
+      long time = System.currentTimeMillis();
+      if (timestamp != null) {
+        try {
+          time = dateFormat.parse(timestamp).getTime();
+        } catch (ParseException e) {
+          throw new IllegalArgumentException(
+              "Unable to parse the specified timestamp " + timestamp, e);
+        }
+      }
+      if (changeModTime ^ changeAccessTime) {
+        long atime = changeModTime ? -1 : time;
+        long mtime = changeAccessTime ? -1 : time;
+        item.fs.setTimes(item.path, mtime, atime);
+      } else {
+        item.fs.setTimes(item.path, time, time);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
index ec9d3c3..d9567b9 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -741,6 +741,38 @@ Usage: `hadoop fs -text <src> `
 
 Takes a source file and outputs the file in text format. The allowed formats are zip and TextRecordInputStream.
 
+touch
+------
+
+Usage: `hadoop fs -touch [-a] [-m] [-t TIMESTAMP] [-c] URI [URI ...]`
+
+Updates the access and modification times of the file specified by the URI to the current time.
+If the file does not exist, then a zero length file is created at URI with current time as the
+timestamp of that URI.
+
+* Use -a option to change only the access time
+* Use -m option to change only the modification time
+* Use -t option to specify timestamp (in format yyyyMMddHHmmss) instead of current time
+* Use -c option to not create file if it does not exist
+
+The timestamp format is as follows
+* yyyy Four digit year (e.g. 2018)
+* MM Two digit month of the year (e.g. 08 for month of August)
+* dd Two digit day of the month (e.g. 01 for first day of the month)
+* HH Two digit hour of the day using 24 hour notation (e.g. 23 stands for 11 pm, 11 stands for 11 am)
+* mm Two digit minutes of the hour
+* ss Two digit seconds of the minute
+e.g. 20180809230000 represents August 9th 2018, 11pm
+
+Example:
+
+* `hadoop fs -touch pathname`
+* `hadoop fs -touch -m -t 20180809230000 pathname`
+* `hadoop fs -touch -t 20180809230000 pathname`
+* `hadoop fs -touch -a pathname`
+
+Exit Code: Returns 0 on success and -1 on error.
+
 touchz
 ------
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java
index 5fe4e39..2e7cb5d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java
@@ -21,7 +21,11 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertThat;
 
+import java.text.ParseException;
+import java.util.Date;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.shell.TouchCommands.Touch;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Before;
@@ -85,4 +89,103 @@ public class TestFsShellTouch {
     assertThat("Expected failed touchz in a non-existent directory",
         shellRun("-touchz", noDirName + "/foo"), is(not(0)));
   }
+
+  @Test
+  public void testTouch() throws Exception {
+    // Ensure newFile2 does not exist
+    final String newFileName = "newFile2";
+    final Path newFile = new Path(newFileName);
+    lfs.delete(newFile, true);
+    assertThat(lfs.exists(newFile), is(false));
+
+    {
+      assertThat(
+          "Expected successful touch on a non-existent file with -c option",
+          shellRun("-touch", "-c", newFileName), is(not(0)));
+      assertThat(lfs.exists(newFile), is(false));
+    }
+
+    {
+      String strTime = formatTimestamp(System.currentTimeMillis());
+      Date dateObj = parseTimestamp(strTime);
+
+      assertThat(
+          "Expected successful touch on a new file with a specified timestamp",
+          shellRun("-touch", "-t", strTime, newFileName), is(0));
+      FileStatus new_status = lfs.getFileStatus(newFile);
+      assertThat(new_status.getAccessTime(), is(dateObj.getTime()));
+      assertThat(new_status.getModificationTime(), is(dateObj.getTime()));
+    }
+
+    FileStatus fstatus = lfs.getFileStatus(newFile);
+
+    {
+      String strTime = formatTimestamp(System.currentTimeMillis());
+      Date dateObj = parseTimestamp(strTime);
+
+      assertThat("Expected successful touch with a specified access time",
+          shellRun("-touch", "-a", "-t", strTime, newFileName), is(0));
+      FileStatus new_status = lfs.getFileStatus(newFile);
+      // Verify if access time is recorded correctly (and modification time
+      // remains unchanged).
+      assertThat(new_status.getAccessTime(), is(dateObj.getTime()));
+      assertThat(new_status.getModificationTime(),
+          is(fstatus.getModificationTime()));
+    }
+
+    fstatus = lfs.getFileStatus(newFile);
+
+    {
+      String strTime = formatTimestamp(System.currentTimeMillis());
+      Date dateObj = parseTimestamp(strTime);
+
+      assertThat(
+          "Expected successful touch with a specified modificatiom time",
+          shellRun("-touch", "-m", "-t", strTime, newFileName), is(0));
+      // Verify if modification time is recorded correctly (and access time
+      // remains unchanged).
+      FileStatus new_status = lfs.getFileStatus(newFile);
+      assertThat(new_status.getAccessTime(), is(fstatus.getAccessTime()));
+      assertThat(new_status.getModificationTime(), is(dateObj.getTime()));
+    }
+
+    {
+      String strTime = formatTimestamp(System.currentTimeMillis());
+      Date dateObj = parseTimestamp(strTime);
+
+      assertThat("Expected successful touch with a specified timestamp",
+          shellRun("-touch", "-t", strTime, newFileName), is(0));
+
+      // Verify if both modification and access times are recorded correctly
+      FileStatus new_status = lfs.getFileStatus(newFile);
+      assertThat(new_status.getAccessTime(), is(dateObj.getTime()));
+      assertThat(new_status.getModificationTime(), is(dateObj.getTime()));
+    }
+
+    {
+      String strTime = formatTimestamp(System.currentTimeMillis());
+      Date dateObj = parseTimestamp(strTime);
+
+      assertThat("Expected successful touch with a specified timestamp",
+          shellRun("-touch", "-a", "-m", "-t", strTime, newFileName), is(0));
+
+      // Verify if both modification and access times are recorded correctly
+      FileStatus new_status = lfs.getFileStatus(newFile);
+      assertThat(new_status.getAccessTime(), is(dateObj.getTime()));
+      assertThat(new_status.getModificationTime(), is(dateObj.getTime()));
+    }
+
+    {
+      assertThat("Expected failed touch with a missing timestamp",
+          shellRun("-touch", "-t", newFileName), is(not(0)));
+    }
+  }
+
+  private String formatTimestamp(long timeInMillis) {
+    return (new Touch()).getDateFormat().format(new Date(timeInMillis));
+  }
+
+  private Date parseTimestamp(String tstamp) throws ParseException {
+    return (new Touch()).getDateFormat().parse(tstamp);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60ffec9f/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index 6a3d53a..1798563 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -840,6 +840,57 @@
     </test>
 
     <test> <!-- TESTED -->
+      <description>help: help for touch</description>
+      <test-commands>
+        <command>-help touch</command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-touch \[-a\] \[-m\] \[-t TIMESTAMP \] \[-c\] &lt;path&gt; \.\.\. :( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*Updates the access and modification times of the file specified by the &lt;path&gt; to( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*the current time. If the file does not exist, then a zero length file is created( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*at &lt;path&gt; with current time as the timestamp of that &lt;path&gt;.( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-a\s+Change only the access time( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-a\s+Change only the access time( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-m\s+Change only the modification time( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-t\s+TIMESTAMP\s+Use specified timestamp \(in format yyyyMMddHHmmss\) instead of</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*current time( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*-c\s+Do not create any files( )*</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
       <description>help: help for touchz</description>
       <test-commands>
         <command>-help touchz</command>


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


[47/50] [abbrv] hadoop git commit: Merge commit '6425ed27ea638da75f656204d6df4adad1d91fe1' into HDFS-12943

Posted by sh...@apache.org.
Merge commit '6425ed27ea638da75f656204d6df4adad1d91fe1' into HDFS-12943


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

Branch: refs/heads/HDFS-12943
Commit: da1822750bb65e282175acd5da2a21fff0f80975
Parents: d53c599 6425ed2
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:17:04 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:24:40 2018 -0700

----------------------------------------------------------------------
 .../federation/router/RouterClientProtocol.java | 1810 ++++++++++++++++++
 .../federation/router/RouterRpcServer.java      | 1364 ++-----------
 2 files changed, 1970 insertions(+), 1204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/da182275/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
index 0000000,f45da3c..7739477
mode 000000,100644..100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
@@@ -1,0 -1,1805 +1,1810 @@@
+ /**
+  * 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.hdfs.server.federation.router;
+ 
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.crypto.CryptoProtocolVersion;
+ import org.apache.hadoop.fs.BatchedRemoteIterator;
+ import org.apache.hadoop.fs.CacheFlag;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.CreateFlag;
+ import org.apache.hadoop.fs.FsServerDefaults;
+ import org.apache.hadoop.fs.Options;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.QuotaUsage;
+ import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.fs.XAttr;
+ import org.apache.hadoop.fs.XAttrSetFlag;
+ import org.apache.hadoop.fs.permission.AclEntry;
+ import org.apache.hadoop.fs.permission.AclStatus;
+ import org.apache.hadoop.fs.permission.FsAction;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hdfs.AddBlockFlag;
+ import org.apache.hadoop.hdfs.DFSConfigKeys;
+ import org.apache.hadoop.hdfs.DFSUtil;
+ import org.apache.hadoop.hdfs.inotify.EventBatchList;
+ import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+ import org.apache.hadoop.hdfs.protocol.DatanodeID;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+ import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+ import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+ import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
+ import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
+ import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+ import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+ import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+ import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
+ import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+ import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+ import org.apache.hadoop.hdfs.server.namenode.NameNode;
+ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+ import org.apache.hadoop.io.EnumSetWritable;
+ import org.apache.hadoop.io.Text;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.security.token.Token;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.util.Collection;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.Iterator;
+ import java.util.LinkedHashMap;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.TreeMap;
+ 
+ /**
+  * Module that implements all the RPC calls in {@link ClientProtocol} in the
+  * {@link RouterRpcServer}.
+  */
+ public class RouterClientProtocol implements ClientProtocol {
+   private static final Logger LOG =
+       LoggerFactory.getLogger(RouterClientProtocol.class.getName());
+ 
+   private final RouterRpcServer rpcServer;
+   private final RouterRpcClient rpcClient;
+   private final FileSubclusterResolver subclusterResolver;
+   private final ActiveNamenodeResolver namenodeResolver;
+ 
+   /** Identifier for the super user. */
+   private final String superUser;
+   /** Identifier for the super group. */
+   private final String superGroup;
+   /** Erasure coding calls. */
+   private final ErasureCoding erasureCoding;
+ 
+   RouterClientProtocol(Configuration conf, RouterRpcServer rpcServer) {
+     this.rpcServer = rpcServer;
+     this.rpcClient = rpcServer.getRPCClient();
+     this.subclusterResolver = rpcServer.getSubclusterResolver();
+     this.namenodeResolver = rpcServer.getNamenodeResolver();
+ 
+     // User and group for reporting
+     this.superUser = System.getProperty("user.name");
+     this.superGroup = conf.get(
+         DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+         DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+     this.erasureCoding = new ErasureCoding(rpcServer);
+   }
+ 
+   @Override
+   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+     return null;
+   }
+ 
+   /**
+    * The the delegation token from each name service.
+    *
+    * @param renewer
+    * @return Name service -> Token.
+    * @throws IOException
+    */
+   public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
+   getDelegationTokens(Text renewer) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+     return null;
+   }
+ 
+   @Override
+   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+     return 0;
+   }
+ 
+   @Override
+   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public LocatedBlocks getBlockLocations(String src, final long offset,
+       final long length) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, false);
+     RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
+         new Class<?>[] {String.class, long.class, long.class},
+         new RemoteParam(), offset, length);
+     return rpcClient.invokeSequential(locations, remoteMethod,
+         LocatedBlocks.class, null);
+   }
+ 
+   @Override
+   public FsServerDefaults getServerDefaults() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     RemoteMethod method = new RemoteMethod("getServerDefaults");
+     String ns = subclusterResolver.getDefaultNamespace();
+     return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
+   }
+ 
+   @Override
+   public HdfsFileStatus create(String src, FsPermission masked,
+       String clientName, EnumSetWritable<CreateFlag> flag,
+       boolean createParent, short replication, long blockSize,
+       CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     if (createParent && isPathAll(src)) {
+       int index = src.lastIndexOf(Path.SEPARATOR);
+       String parent = src.substring(0, index);
+       LOG.debug("Creating {} requires creating parent {}", src, parent);
+       FsPermission parentPermissions = getParentPermission(masked);
+       boolean success = mkdirs(parent, parentPermissions, createParent);
+       if (!success) {
+         // This shouldn't happen as mkdirs returns true or exception
+         LOG.error("Couldn't create parents for {}", src);
+       }
+     }
+ 
+     RemoteLocation createLocation = rpcServer.getCreateLocation(src);
+     RemoteMethod method = new RemoteMethod("create",
+         new Class<?>[] {String.class, FsPermission.class, String.class,
+             EnumSetWritable.class, boolean.class, short.class,
+             long.class, CryptoProtocolVersion[].class,
+             String.class},
+         createLocation.getDest(), masked, clientName, flag, createParent,
+         replication, blockSize, supportedVersions, ecPolicyName);
+     return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
+   }
+ 
+   @Override
+   public LastBlockWithStatus append(String src, final String clientName,
+       final EnumSetWritable<CreateFlag> flag) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("append",
+         new Class<?>[] {String.class, String.class, EnumSetWritable.class},
+         new RemoteParam(), clientName, flag);
+     return rpcClient.invokeSequential(
+         locations, method, LastBlockWithStatus.class, null);
+   }
+ 
+   @Override
+   public boolean recoverLease(String src, String clientName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("recoverLease",
+         new Class<?>[] {String.class, String.class}, new RemoteParam(),
+         clientName);
+     Object result = rpcClient.invokeSequential(
+         locations, method, Boolean.class, Boolean.TRUE);
+     return (boolean) result;
+   }
+ 
+   @Override
+   public boolean setReplication(String src, short replication)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setReplication",
+         new Class<?>[] {String.class, short.class}, new RemoteParam(),
+         replication);
+     Object result = rpcClient.invokeSequential(
+         locations, method, Boolean.class, Boolean.TRUE);
+     return (boolean) result;
+   }
+ 
+   @Override
+   public void setStoragePolicy(String src, String policyName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setStoragePolicy",
+         new Class<?>[] {String.class, String.class},
+         new RemoteParam(), policyName);
+     rpcClient.invokeSequential(locations, method, null, null);
+   }
+ 
+   @Override
+   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     RemoteMethod method = new RemoteMethod("getStoragePolicies");
+     String ns = subclusterResolver.getDefaultNamespace();
+     return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
+   }
+ 
+   @Override
+   public void setPermission(String src, FsPermission permissions)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setPermission",
+         new Class<?>[] {String.class, FsPermission.class},
+         new RemoteParam(), permissions);
+     if (isPathAll(src)) {
+       rpcClient.invokeConcurrent(locations, method);
+     } else {
+       rpcClient.invokeSequential(locations, method);
+     }
+   }
+ 
+   @Override
+   public void setOwner(String src, String username, String groupname)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setOwner",
+         new Class<?>[] {String.class, String.class, String.class},
+         new RemoteParam(), username, groupname);
+     if (isPathAll(src)) {
+       rpcClient.invokeConcurrent(locations, method);
+     } else {
+       rpcClient.invokeSequential(locations, method);
+     }
+   }
+ 
+   /**
+    * Excluded and favored nodes are not verified and will be ignored by
+    * placement policy if they are not in the same nameservice as the file.
+    */
+   @Override
+   public LocatedBlock addBlock(String src, String clientName,
+       ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
+       String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("addBlock",
+         new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+             DatanodeInfo[].class, long.class, String[].class,
+             EnumSet.class},
+         new RemoteParam(), clientName, previous, excludedNodes, fileId,
+         favoredNodes, addBlockFlags);
+     // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
+     return rpcClient.invokeSequential(
+         locations, method, LocatedBlock.class, null);
+   }
+ 
+   /**
+    * Excluded nodes are not verified and will be ignored by placement if they
+    * are not in the same nameservice as the file.
+    */
+   @Override
+   public LocatedBlock getAdditionalDatanode(final String src, final long fileId,
+       final ExtendedBlock blk, final DatanodeInfo[] existings,
+       final String[] existingStorageIDs, final DatanodeInfo[] excludes,
+       final int numAdditionalNodes, final String clientName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
+         new Class<?>[] {String.class, long.class, ExtendedBlock.class,
+             DatanodeInfo[].class, String[].class,
+             DatanodeInfo[].class, int.class, String.class},
+         new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
+         numAdditionalNodes, clientName);
+     return rpcClient.invokeSequential(
+         locations, method, LocatedBlock.class, null);
+   }
+ 
+   @Override
+   public void abandonBlock(ExtendedBlock b, long fileId, String src,
+       String holder) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     RemoteMethod method = new RemoteMethod("abandonBlock",
+         new Class<?>[] {ExtendedBlock.class, long.class, String.class,
+             String.class},
+         b, fileId, new RemoteParam(), holder);
+     rpcClient.invokeSingle(b, method);
+   }
+ 
+   @Override
+   public boolean complete(String src, String clientName, ExtendedBlock last,
+       long fileId) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("complete",
+         new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+             long.class},
+         new RemoteParam(), clientName, last, fileId);
+     // Complete can return true/false, so don't expect a result
+     return rpcClient.invokeSequential(locations, method, Boolean.class, null);
+   }
+ 
+   @Override
+   public LocatedBlock updateBlockForPipeline(
+       ExtendedBlock block, String clientName) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
+         new Class<?>[] {ExtendedBlock.class, String.class},
+         block, clientName);
+     return (LocatedBlock) rpcClient.invokeSingle(block, method);
+   }
+ 
+   /**
+    * Datanode are not verified to be in the same nameservice as the old block.
+    * TODO This may require validation.
+    */
+   @Override
+   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     RemoteMethod method = new RemoteMethod("updatePipeline",
+         new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
+             DatanodeID[].class, String[].class},
+         clientName, oldBlock, newBlock, newNodes, newStorageIDs);
+     rpcClient.invokeSingle(oldBlock, method);
+   }
+ 
+   @Override
+   public long getPreferredBlockSize(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(locations, method, Long.class, null);
+   }
+ 
+   @Deprecated
+   @Override
+   public boolean rename(final String src, final String dst)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> srcLocations =
+         rpcServer.getLocationsForPath(src, true, false);
+     // srcLocations may be trimmed by getRenameDestinations()
+     final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+     RemoteParam dstParam = getRenameDestinations(locs, dst);
+     if (locs.isEmpty()) {
+       throw new IOException(
+           "Rename of " + src + " to " + dst + " is not allowed," +
+               " no eligible destination in the same namespace was found.");
+     }
+     RemoteMethod method = new RemoteMethod("rename",
+         new Class<?>[] {String.class, String.class},
+         new RemoteParam(), dstParam);
+     return rpcClient.invokeSequential(locs, method, Boolean.class,
+         Boolean.TRUE);
+   }
+ 
+   @Override
+   public void rename2(final String src, final String dst,
+       final Options.Rename... options) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> srcLocations =
+         rpcServer.getLocationsForPath(src, true, false);
+     // srcLocations may be trimmed by getRenameDestinations()
+     final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+     RemoteParam dstParam = getRenameDestinations(locs, dst);
+     if (locs.isEmpty()) {
+       throw new IOException(
+           "Rename of " + src + " to " + dst + " is not allowed," +
+               " no eligible destination in the same namespace was found.");
+     }
+     RemoteMethod method = new RemoteMethod("rename2",
+         new Class<?>[] {String.class, String.class, options.getClass()},
+         new RemoteParam(), dstParam, options);
+     rpcClient.invokeSequential(locs, method, null, null);
+   }
+ 
+   @Override
+   public void concat(String trg, String[] src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // See if the src and target files are all in the same namespace
+     LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
+     if (targetBlocks == null) {
+       throw new IOException("Cannot locate blocks for target file - " + trg);
+     }
+     LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
+     String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
+     for (String source : src) {
+       LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
+       if (sourceBlocks == null) {
+         throw new IOException(
+             "Cannot located blocks for source file " + source);
+       }
+       String sourceBlockPoolId =
+           sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
+       if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
+         throw new IOException("Cannot concatenate source file " + source
+             + " because it is located in a different namespace"
+             + " with block pool id " + sourceBlockPoolId
+             + " from the target file with block pool id "
+             + targetBlockPoolId);
+       }
+     }
+ 
+     // Find locations in the matching namespace.
+     final RemoteLocation targetDestination =
+         rpcServer.getLocationForPath(trg, true, targetBlockPoolId);
+     String[] sourceDestinations = new String[src.length];
+     for (int i = 0; i < src.length; i++) {
+       String sourceFile = src[i];
+       RemoteLocation location =
+           rpcServer.getLocationForPath(sourceFile, true, targetBlockPoolId);
+       sourceDestinations[i] = location.getDest();
+     }
+     // Invoke
+     RemoteMethod method = new RemoteMethod("concat",
+         new Class<?>[] {String.class, String[].class},
+         targetDestination.getDest(), sourceDestinations);
+     rpcClient.invokeSingle(targetDestination, method);
+   }
+ 
+   @Override
+   public boolean truncate(String src, long newLength, String clientName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("truncate",
+         new Class<?>[] {String.class, long.class, String.class},
+         new RemoteParam(), newLength, clientName);
+     return rpcClient.invokeSequential(locations, method, Boolean.class,
+         Boolean.TRUE);
+   }
+ 
+   @Override
+   public boolean delete(String src, boolean recursive) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true, false);
+     RemoteMethod method = new RemoteMethod("delete",
+         new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+         recursive);
+     if (isPathAll(src)) {
+       return rpcClient.invokeAll(locations, method);
+     } else {
+       return rpcClient.invokeSequential(locations, method,
+           Boolean.class, Boolean.TRUE);
+     }
+   }
+ 
+   @Override
+   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("mkdirs",
+         new Class<?>[] {String.class, FsPermission.class, boolean.class},
+         new RemoteParam(), masked, createParent);
+ 
+     // Create in all locations
+     if (isPathAll(src)) {
+       return rpcClient.invokeAll(locations, method);
+     }
+ 
+     if (locations.size() > 1) {
+       // Check if this directory already exists
+       try {
+         HdfsFileStatus fileStatus = getFileInfo(src);
+         if (fileStatus != null) {
+           // When existing, the NN doesn't return an exception; return true
+           return true;
+         }
+       } catch (IOException ioe) {
+         // Can't query if this file exists or not.
+         LOG.error("Error requesting file info for path {} while proxing mkdirs",
+             src, ioe);
+       }
+     }
+ 
+     RemoteLocation firstLocation = locations.get(0);
+     return (boolean) rpcClient.invokeSingle(firstLocation, method);
+   }
+ 
+   @Override
+   public void renewLease(String clientName) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     RemoteMethod method = new RemoteMethod("renewLease",
+         new Class<?>[] {String.class}, clientName);
+     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     rpcClient.invokeConcurrent(nss, method, false, false);
+   }
+ 
+   @Override
+   public DirectoryListing getListing(String src, byte[] startAfter,
+       boolean needLocation) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // Locate the dir and fetch the listing
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("getListing",
+         new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
+         new RemoteParam(), startAfter, needLocation);
+     Map<RemoteLocation, DirectoryListing> listings =
+         rpcClient.invokeConcurrent(
+             locations, method, false, false, DirectoryListing.class);
+ 
+     Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
+     int totalRemainingEntries = 0;
+     int remainingEntries = 0;
+     boolean namenodeListingExists = false;
+     if (listings != null) {
+       // Check the subcluster listing with the smallest name
+       String lastName = null;
+       for (Map.Entry<RemoteLocation, DirectoryListing> entry :
+           listings.entrySet()) {
+         RemoteLocation location = entry.getKey();
+         DirectoryListing listing = entry.getValue();
+         if (listing == null) {
+           LOG.debug("Cannot get listing from {}", location);
+         } else {
+           totalRemainingEntries += listing.getRemainingEntries();
+           HdfsFileStatus[] partialListing = listing.getPartialListing();
+           int length = partialListing.length;
+           if (length > 0) {
+             HdfsFileStatus lastLocalEntry = partialListing[length-1];
+             String lastLocalName = lastLocalEntry.getLocalName();
+             if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
+               lastName = lastLocalName;
+             }
+           }
+         }
+       }
+ 
+       // Add existing entries
+       for (Object value : listings.values()) {
+         DirectoryListing listing = (DirectoryListing) value;
+         if (listing != null) {
+           namenodeListingExists = true;
+           for (HdfsFileStatus file : listing.getPartialListing()) {
+             String filename = file.getLocalName();
+             if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
+               // Discarding entries further than the lastName
+               remainingEntries++;
+             } else {
+               nnListing.put(filename, file);
+             }
+           }
+           remainingEntries += listing.getRemainingEntries();
+         }
+       }
+     }
+ 
+     // Add mount points at this level in the tree
+     final List<String> children = subclusterResolver.getMountPoints(src);
+     if (children != null) {
+       // Get the dates for each mount point
+       Map<String, Long> dates = getMountPointDates(src);
+ 
+       // Create virtual folder with the mount name
+       for (String child : children) {
+         long date = 0;
+         if (dates != null && dates.containsKey(child)) {
+           date = dates.get(child);
+         }
+         // TODO add number of children
+         HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
+ 
+         // This may overwrite existing listing entries with the mount point
+         // TODO don't add if already there?
+         nnListing.put(child, dirStatus);
+       }
+     }
+ 
+     if (!namenodeListingExists && nnListing.size() == 0) {
+       // NN returns a null object if the directory cannot be found and has no
+       // listing. If we didn't retrieve any NN listing data, and there are no
+       // mount points here, return null.
+       return null;
+     }
+ 
+     // Generate combined listing
+     HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
+     combinedData = nnListing.values().toArray(combinedData);
+     return new DirectoryListing(combinedData, remainingEntries);
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileInfo(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getFileInfo",
+         new Class<?>[] {String.class}, new RemoteParam());
+ 
+     HdfsFileStatus ret = null;
+     // If it's a directory, we check in all locations
+     if (isPathAll(src)) {
+       ret = getFileInfoAll(locations, method);
+     } else {
+       // Check for file information sequentially
+       ret = rpcClient.invokeSequential(
+           locations, method, HdfsFileStatus.class, null);
+     }
+ 
+     // If there is no real path, check mount points
+     if (ret == null) {
+       List<String> children = subclusterResolver.getMountPoints(src);
+       if (children != null && !children.isEmpty()) {
+         Map<String, Long> dates = getMountPointDates(src);
+         long date = 0;
+         if (dates != null && dates.containsKey(src)) {
+           date = dates.get(src);
+         }
+         ret = getMountPointStatus(src, children.size(), date);
+       }
+     }
+ 
+     return ret;
+   }
+ 
+   @Override
+   public boolean isFileClosed(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("isFileClosed",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(locations, method, Boolean.class,
+         Boolean.TRUE);
+   }
+ 
+   @Override
+   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getFileLinkInfo",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(locations, method, HdfsFileStatus.class,
+         null);
+   }
+ 
+   @Override
+   public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+       boolean needBlockToken) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
+         new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+         needBlockToken);
+     return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
+         locations, method, HdfsFileStatus.class, null);
+   }
+ 
+   @Override
+   public long[] getStats() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("getStats");
+     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, long[]> results =
+         rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
+     long[] combinedData = new long[STATS_ARRAY_LENGTH];
+     for (long[] data : results.values()) {
+       for (int i = 0; i < combinedData.length && i < data.length; i++) {
+         if (data[i] >= 0) {
+           combinedData[i] += data[i];
+         }
+       }
+     }
+     return combinedData;
+   }
+ 
+   @Override
+   public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+     return rpcServer.getDatanodeReport(type, true, 0);
+   }
+ 
+   @Override
+   public DatanodeStorageReport[] getDatanodeStorageReport(
+       HdfsConstants.DatanodeReportType type) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     Map<String, DatanodeStorageReport[]> dnSubcluster =
+         rpcServer.getDatanodeStorageReportMap(type);
+ 
+     // Avoid repeating machines in multiple subclusters
+     Map<String, DatanodeStorageReport> datanodesMap = new LinkedHashMap<>();
+     for (DatanodeStorageReport[] dns : dnSubcluster.values()) {
+       for (DatanodeStorageReport dn : dns) {
+         DatanodeInfo dnInfo = dn.getDatanodeInfo();
+         String nodeId = dnInfo.getXferAddr();
+         if (!datanodesMap.containsKey(nodeId)) {
+           datanodesMap.put(nodeId, dn);
+         }
+         // TODO merge somehow, right now it just takes the first one
+       }
+     }
+ 
+     Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
+     DatanodeStorageReport[] combinedData =
+         new DatanodeStorageReport[datanodes.size()];
+     combinedData = datanodes.toArray(combinedData);
+     return combinedData;
+   }
+ 
+   @Override
+   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
+       boolean isChecked) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // Set safe mode in all the name spaces
+     RemoteMethod method = new RemoteMethod("setSafeMode",
+         new Class<?>[] {HdfsConstants.SafeModeAction.class, boolean.class},
+         action, isChecked);
+     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, Boolean> results =
+         rpcClient.invokeConcurrent(
+             nss, method, true, !isChecked, Boolean.class);
+ 
+     // We only report true if all the name space are in safe mode
+     int numSafemode = 0;
+     for (boolean safemode : results.values()) {
+       if (safemode) {
+         numSafemode++;
+       }
+     }
+     return numSafemode == results.size();
+   }
+ 
+   @Override
+   public boolean restoreFailedStorage(String arg) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("restoreFailedStorage",
+         new Class<?>[] {String.class}, arg);
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, Boolean> ret =
+         rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class);
+ 
+     boolean success = true;
+     for (boolean s : ret.values()) {
+       if (!s) {
+         success = false;
+         break;
+       }
+     }
+     return success;
+   }
+ 
+   @Override
+   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("saveNamespace",
+         new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, Boolean> ret =
+         rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
+ 
+     boolean success = true;
+     for (boolean s : ret.values()) {
+       if (!s) {
+         success = false;
+         break;
+       }
+     }
+     return success;
+   }
+ 
+   @Override
+   public long rollEdits() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, Long> ret =
+         rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+ 
+     // Return the maximum txid
+     long txid = 0;
+     for (long t : ret.values()) {
+       if (t > txid) {
+         txid = t;
+       }
+     }
+     return txid;
+   }
+ 
+   @Override
+   public void refreshNodes() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     rpcClient.invokeConcurrent(nss, method, true, true);
+   }
+ 
+   @Override
+   public void finalizeUpgrade() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("finalizeUpgrade",
+         new Class<?>[] {});
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     rpcClient.invokeConcurrent(nss, method, true, false);
+   }
+ 
+   @Override
+   public boolean upgradeStatus() throws IOException {
+     String methodName = RouterRpcServer.getMethodName();
+     throw new UnsupportedOperationException(
+         "Operation \"" + methodName + "\" is not supported");
+   }
+ 
+   @Override
+   public RollingUpgradeInfo rollingUpgrade(HdfsConstants.RollingUpgradeAction action)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     RemoteMethod method = new RemoteMethod("rollingUpgrade",
+         new Class<?>[] {HdfsConstants.RollingUpgradeAction.class}, action);
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+         rpcClient.invokeConcurrent(
+             nss, method, true, false, RollingUpgradeInfo.class);
+ 
+     // Return the first rolling upgrade info
+     RollingUpgradeInfo info = null;
+     for (RollingUpgradeInfo infoNs : ret.values()) {
+       if (info == null && infoNs != null) {
+         info = infoNs;
+       }
+     }
+     return info;
+   }
+ 
+   @Override
+   public void metaSave(String filename) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("metaSave",
+         new Class<?>[] {String.class}, filename);
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     rpcClient.invokeConcurrent(nss, method, true, false);
+   }
+ 
+   @Override
+   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(path, false);
+     RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
+         new Class<?>[] {String.class, String.class},
+         new RemoteParam(), cookie);
+     return rpcClient.invokeSequential(
+         locations, method, CorruptFileBlocks.class, null);
+   }
+ 
+   @Override
+   public void setBalancerBandwidth(long bandwidth) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+ 
+     RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
+         new Class<?>[] {Long.class}, bandwidth);
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     rpcClient.invokeConcurrent(nss, method, true, false);
+   }
+ 
+   @Override
+   public ContentSummary getContentSummary(String path) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // Get the summaries from regular files
+     Collection<ContentSummary> summaries = new LinkedList<>();
+     FileNotFoundException notFoundException = null;
+     try {
+       final List<RemoteLocation> locations =
+           rpcServer.getLocationsForPath(path, false);
+       RemoteMethod method = new RemoteMethod("getContentSummary",
+           new Class<?>[] {String.class}, new RemoteParam());
+       Map<RemoteLocation, ContentSummary> results =
+           rpcClient.invokeConcurrent(
+               locations, method, false, false, ContentSummary.class);
+       summaries.addAll(results.values());
+     } catch (FileNotFoundException e) {
+       notFoundException = e;
+     }
+ 
+     // Add mount points at this level in the tree
+     final List<String> children = subclusterResolver.getMountPoints(path);
+     if (children != null) {
+       for (String child : children) {
+         Path childPath = new Path(path, child);
+         try {
+           ContentSummary mountSummary = getContentSummary(childPath.toString());
+           if (mountSummary != null) {
+             summaries.add(mountSummary);
+           }
+         } catch (Exception e) {
+           LOG.error("Cannot get content summary for mount {}: {}",
+               childPath, e.getMessage());
+         }
+       }
+     }
+ 
+     // Throw original exception if no original nor mount points
+     if (summaries.isEmpty() && notFoundException != null) {
+       throw notFoundException;
+     }
+ 
+     return aggregateContentSummary(summaries);
+   }
+ 
+   @Override
+   public void fsync(String src, long fileId, String clientName,
+       long lastBlockLength) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("fsync",
+         new Class<?>[] {String.class, long.class, String.class, long.class },
+         new RemoteParam(), fileId, clientName, lastBlockLength);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public void setTimes(String src, long mtime, long atime) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setTimes",
+         new Class<?>[] {String.class, long.class, long.class},
+         new RemoteParam(), mtime, atime);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public void createSymlink(String target, String link, FsPermission dirPerms,
+       boolean createParent) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO Verify that the link location is in the same NS as the targets
+     final List<RemoteLocation> targetLocations =
+         rpcServer.getLocationsForPath(target, true);
+     final List<RemoteLocation> linkLocations =
+         rpcServer.getLocationsForPath(link, true);
+     RemoteLocation linkLocation = linkLocations.get(0);
+     RemoteMethod method = new RemoteMethod("createSymlink",
+         new Class<?>[] {String.class, String.class, FsPermission.class,
+             boolean.class},
+         new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
+     rpcClient.invokeSequential(targetLocations, method);
+   }
+ 
+   @Override
+   public String getLinkTarget(String path) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(path, true);
+     RemoteMethod method = new RemoteMethod("getLinkTarget",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(locations, method, String.class, null);
+   }
+ 
+   @Override // Client Protocol
+   public void allowSnapshot(String snapshotRoot) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override // Client Protocol
+   public void disallowSnapshot(String snapshot) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+       String snapshotNewName) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+       String earlierSnapshotName, String laterSnapshotName) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public SnapshotDiffReportListing getSnapshotDiffReportListing(
+       String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+       byte[] startPath, int index) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public long addCacheDirective(CacheDirectiveInfo path,
+       EnumSet<CacheFlag> flags) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+     return 0;
+   }
+ 
+   @Override
+   public void modifyCacheDirective(CacheDirectiveInfo directive,
+       EnumSet<CacheFlag> flags) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public void removeCacheDirective(long id) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+       long prevId, CacheDirectiveInfo filter) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public void addCachePool(CachePoolInfo info) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public void modifyCachePool(CachePoolInfo info) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public void removeCachePool(String cachePoolName) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("modifyAclEntries",
+         new Class<?>[] {String.class, List.class},
+         new RemoteParam(), aclSpec);
+     rpcClient.invokeSequential(locations, method, null, null);
+   }
+ 
+   @Override
+   public void removeAclEntries(String src, List<AclEntry> aclSpec)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("removeAclEntries",
+         new Class<?>[] {String.class, List.class},
+         new RemoteParam(), aclSpec);
+     rpcClient.invokeSequential(locations, method, null, null);
+   }
+ 
+   @Override
+   public void removeDefaultAcl(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("removeDefaultAcl",
+         new Class<?>[] {String.class}, new RemoteParam());
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public void removeAcl(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("removeAcl",
+         new Class<?>[] {String.class}, new RemoteParam());
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod(
+         "setAcl", new Class<?>[] {String.class, List.class},
+         new RemoteParam(), aclSpec);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public AclStatus getAclStatus(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getAclStatus",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(locations, method, AclStatus.class, null);
+   }
+ 
+   @Override
+   public void createEncryptionZone(String src, String keyName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("createEncryptionZone",
+         new Class<?>[] {String.class, String.class},
+         new RemoteParam(), keyName);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public EncryptionZone getEZForPath(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getEZForPath",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return rpcClient.invokeSequential(
+         locations, method, EncryptionZone.class, null);
+   }
+ 
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public void reencryptEncryptionZone(String zone, HdfsConstants.ReencryptAction action)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(
+       long prevId) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("setXAttr",
+         new Class<?>[] {String.class, XAttr.class, EnumSet.class},
+         new RemoteParam(), xAttr, flag);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @SuppressWarnings("unchecked")
+   @Override
+   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("getXAttrs",
+         new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
+     return (List<XAttr>) rpcClient.invokeSequential(
+         locations, method, List.class, null);
+   }
+ 
+   @SuppressWarnings("unchecked")
+   @Override
+   public List<XAttr> listXAttrs(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, false);
+     RemoteMethod method = new RemoteMethod("listXAttrs",
+         new Class<?>[] {String.class}, new RemoteParam());
+     return (List<XAttr>) rpcClient.invokeSequential(
+         locations, method, List.class, null);
+   }
+ 
+   @Override
+   public void removeXAttr(String src, XAttr xAttr) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(src, true);
+     RemoteMethod method = new RemoteMethod("removeXAttr",
+         new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public void checkAccess(String path, FsAction mode) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     // TODO handle virtual directories
+     final List<RemoteLocation> locations =
+         rpcServer.getLocationsForPath(path, true);
+     RemoteMethod method = new RemoteMethod("checkAccess",
+         new Class<?>[] {String.class, FsAction.class},
+         new RemoteParam(), mode);
+     rpcClient.invokeSequential(locations, method);
+   }
+ 
+   @Override
+   public long getCurrentEditLogTxid() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ);
+ 
+     RemoteMethod method = new RemoteMethod(
+         "getCurrentEditLogTxid", new Class<?>[] {});
+     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+     Map<FederationNamespaceInfo, Long> ret =
+         rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+ 
+     // Return the maximum txid
+     long txid = 0;
+     for (long t : ret.values()) {
+       if (t > txid) {
+         txid = t;
+       }
+     }
+     return txid;
+   }
+ 
+   @Override
+   public EventBatchList getEditsFromTxid(long txid) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public DataEncryptionKey getDataEncryptionKey() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public String createSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+     return null;
+   }
+ 
+   @Override
+   public void deleteSnapshot(String snapshotRoot, String snapshotName)
+       throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+       StorageType type) throws IOException {
+     rpcServer.getQuotaModule()
+         .setQuota(path, namespaceQuota, storagespaceQuota, type);
+   }
+ 
+   @Override
+   public QuotaUsage getQuotaUsage(String path) throws IOException {
+     return rpcServer.getQuotaModule().getQuotaUsage(path);
+   }
+ 
+   @Override
+   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+ 
+     // Block pool id -> blocks
+     Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
+     for (LocatedBlock block : blocks) {
+       String bpId = block.getBlock().getBlockPoolId();
+       List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
+       if (bpBlocks == null) {
+         bpBlocks = new LinkedList<>();
+         blockLocations.put(bpId, bpBlocks);
+       }
+       bpBlocks.add(block);
+     }
+ 
+     // Invoke each block pool
+     for (Map.Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
+       String bpId = entry.getKey();
+       List<LocatedBlock> bpBlocks = entry.getValue();
+ 
+       LocatedBlock[] bpBlocksArray =
+           bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
+       RemoteMethod method = new RemoteMethod("reportBadBlocks",
+           new Class<?>[] {LocatedBlock[].class},
+           new Object[] {bpBlocksArray});
+       rpcClient.invokeSingleBlockPool(bpId, method);
+     }
+   }
+ 
+   @Override
+   public void unsetStoragePolicy(String src) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   @Override
+   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
+   public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+       throws IOException {
+     return erasureCoding.getErasureCodingPolicies();
+   }
+ 
+   @Override
+   public Map<String, String> getErasureCodingCodecs() throws IOException {
+     return erasureCoding.getErasureCodingCodecs();
+   }
+ 
+   @Override
+   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+       ErasureCodingPolicy[] policies) throws IOException {
+     return erasureCoding.addErasureCodingPolicies(policies);
+   }
+ 
+   @Override
+   public void removeErasureCodingPolicy(String ecPolicyName)
+       throws IOException {
+     erasureCoding.removeErasureCodingPolicy(ecPolicyName);
+   }
+ 
+   @Override
+   public void disableErasureCodingPolicy(String ecPolicyName)
+       throws IOException {
+     erasureCoding.disableErasureCodingPolicy(ecPolicyName);
+   }
+ 
+   @Override
+   public void enableErasureCodingPolicy(String ecPolicyName)
+       throws IOException {
+     erasureCoding.enableErasureCodingPolicy(ecPolicyName);
+   }
+ 
+   @Override
+   public ErasureCodingPolicy getErasureCodingPolicy(String src)
+       throws IOException {
+     return erasureCoding.getErasureCodingPolicy(src);
+   }
+ 
+   @Override
+   public void setErasureCodingPolicy(String src, String ecPolicyName)
+       throws IOException {
+     erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+   }
+ 
+   @Override
+   public void unsetErasureCodingPolicy(String src) throws IOException {
+     erasureCoding.unsetErasureCodingPolicy(src);
+   }
+ 
+   @Override
+   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+     return erasureCoding.getECBlockGroupStats();
+   }
+ 
+   @Override
+   public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Deprecated
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
+       throws IOException {
+     return listOpenFiles(prevId, EnumSet.of(OpenFilesIterator.OpenFilesType.ALL_OPEN_FILES),
+         OpenFilesIterator.FILTER_PATH_DEFAULT);
+   }
+ 
+   @Override
+   public BatchedRemoteIterator.BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+       EnumSet<OpenFilesIterator.OpenFilesType> openFilesTypes, String path) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+     return null;
+   }
+ 
+   @Override
++  public void msync() throws IOException {
++    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
++  }
++
++  @Override
+   public void satisfyStoragePolicy(String path) throws IOException {
+     rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+   }
+ 
+   /**
+    * Determines combinations of eligible src/dst locations for a rename. A
+    * rename cannot change the namespace. Renames are only allowed if there is an
+    * eligible dst location in the same namespace as the source.
+    *
+    * @param srcLocations List of all potential source destinations where the
+    *          path may be located. On return this list is trimmed to include
+    *          only the paths that have corresponding destinations in the same
+    *          namespace.
+    * @param dst The destination path
+    * @return A map of all eligible source namespaces and their corresponding
+    *         replacement value.
+    * @throws IOException If the dst paths could not be determined.
+    */
+   private RemoteParam getRenameDestinations(
+       final List<RemoteLocation> srcLocations, final String dst)
+       throws IOException {
+ 
+     final List<RemoteLocation> dstLocations =
+         rpcServer.getLocationsForPath(dst, true);
+     final Map<RemoteLocation, String> dstMap = new HashMap<>();
+ 
+     Iterator<RemoteLocation> iterator = srcLocations.iterator();
+     while (iterator.hasNext()) {
+       RemoteLocation srcLocation = iterator.next();
+       RemoteLocation eligibleDst =
+           getFirstMatchingLocation(srcLocation, dstLocations);
+       if (eligibleDst != null) {
+         // Use this dst for this source location
+         dstMap.put(srcLocation, eligibleDst.getDest());
+       } else {
+         // This src destination is not valid, remove from the source list
+         iterator.remove();
+       }
+     }
+     return new RemoteParam(dstMap);
+   }
+ 
+   /**
+    * Get first matching location.
+    *
+    * @param location Location we are looking for.
+    * @param locations List of locations.
+    * @return The first matchin location in the list.
+    */
+   private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
+       List<RemoteLocation> locations) {
+     for (RemoteLocation loc : locations) {
+       if (loc.getNameserviceId().equals(location.getNameserviceId())) {
+         // Return first matching location
+         return loc;
+       }
+     }
+     return null;
+   }
+ 
+   /**
+    * Aggregate content summaries for each subcluster.
+    *
+    * @param summaries Collection of individual summaries.
+    * @return Aggregated content summary.
+    */
+   private ContentSummary aggregateContentSummary(
+       Collection<ContentSummary> summaries) {
+     if (summaries.size() == 1) {
+       return summaries.iterator().next();
+     }
+ 
+     long length = 0;
+     long fileCount = 0;
+     long directoryCount = 0;
+     long quota = 0;
+     long spaceConsumed = 0;
+     long spaceQuota = 0;
+ 
+     for (ContentSummary summary : summaries) {
+       length += summary.getLength();
+       fileCount += summary.getFileCount();
+       directoryCount += summary.getDirectoryCount();
+       quota += summary.getQuota();
+       spaceConsumed += summary.getSpaceConsumed();
+       spaceQuota += summary.getSpaceQuota();
+     }
+ 
+     ContentSummary ret = new ContentSummary.Builder()
+         .length(length)
+         .fileCount(fileCount)
+         .directoryCount(directoryCount)
+         .quota(quota)
+         .spaceConsumed(spaceConsumed)
+         .spaceQuota(spaceQuota)
+         .build();
+     return ret;
+   }
+ 
+   /**
+    * Get the file info from all the locations.
+    *
+    * @param locations Locations to check.
+    * @param method The file information method to run.
+    * @return The first file info if it's a file, the directory if it's
+    *         everywhere.
+    * @throws IOException If all the locations throw an exception.
+    */
+   private HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
+       final RemoteMethod method) throws IOException {
+ 
+     // Get the file info from everybody
+     Map<RemoteLocation, HdfsFileStatus> results =
+         rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
+ 
+     // We return the first file
+     HdfsFileStatus dirStatus = null;
+     for (RemoteLocation loc : locations) {
+       HdfsFileStatus fileStatus = results.get(loc);
+       if (fileStatus != null) {
+         if (!fileStatus.isDirectory()) {
+           return fileStatus;
+         } else if (dirStatus == null) {
+           dirStatus = fileStatus;
+         }
+       }
+     }
+     return dirStatus;
+   }
+ 
+   /**
+    * Get the permissions for the parent of a child with given permissions.
+    * Add implicit u+wx permission for parent. This is based on
+    * @{FSDirMkdirOp#addImplicitUwx}.
+    * @param mask The permission mask of the child.
+    * @return The permission mask of the parent.
+    */
+   private static FsPermission getParentPermission(final FsPermission mask) {
+     FsPermission ret = new FsPermission(
+         mask.getUserAction().or(FsAction.WRITE_EXECUTE),
+         mask.getGroupAction(),
+         mask.getOtherAction());
+     return ret;
+   }
+ 
+   /**
+    * Check if a path should be in all subclusters.
+    *
+    * @param path Path to check.
+    * @return If a path should be in all subclusters.
+    */
+   private boolean isPathAll(final String path) {
+     if (subclusterResolver instanceof MountTableResolver) {
+       try {
+         MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+         MountTable entry = mountTable.getMountPoint(path);
+         if (entry != null) {
+           return entry.isAll();
+         }
+       } catch (IOException e) {
+         LOG.error("Cannot get mount point", e);
+       }
+     }
+     return false;
+   }
+ 
+   /**
+    * Create a new file status for a mount point.
+    *
+    * @param name Name of the mount point.
+    * @param childrenNum Number of children.
+    * @param date Map with the dates.
+    * @return New HDFS file status representing a mount point.
+    */
+   private HdfsFileStatus getMountPointStatus(
+       String name, int childrenNum, long date) {
+     long modTime = date;
+     long accessTime = date;
+     FsPermission permission = FsPermission.getDirDefault();
+     String owner = this.superUser;
+     String group = this.superGroup;
+     try {
+       // TODO support users, it should be the user for the pointed folder
+       UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+       owner = ugi.getUserName();
+       group = ugi.getPrimaryGroupName();
+     } catch (IOException e) {
+       LOG.error("Cannot get the remote user: {}", e.getMessage());
+     }
+     long inodeId = 0;
+     return new HdfsFileStatus.Builder()
+         .isdir(true)
+         .mtime(modTime)
+         .atime(accessTime)
+         .perm(permission)
+         .owner(owner)
+         .group(group)
+         .symlink(new byte[0])
+         .path(DFSUtil.string2Bytes(name))
+         .fileId(inodeId)
+         .children(childrenNum)
+         .build();
+   }
+ 
+   /**
+    * Get the modification dates for mount points.
+    *
+    * @param path Name of the path to start checking dates from.
+    * @return Map with the modification dates for all sub-entries.
+    */
+   private Map<String, Long> getMountPointDates(String path) {
+     Map<String, Long> ret = new TreeMap<>();
+     if (subclusterResolver instanceof MountTableResolver) {
+       try {
+         final List<String> children = subclusterResolver.getMountPoints(path);
+         for (String child : children) {
+           Long modTime = getModifiedTime(ret, path, child);
+           ret.put(child, modTime);
+         }
+       } catch (IOException e) {
+         LOG.error("Cannot get mount point", e);
+       }
+     }
+     return ret;
+   }
+ 
+   /**
+    * Get modified time for child. If the child is present in mount table it
+    * will return the modified time. If the child is not present but subdirs of
+    * this child are present then it will return latest modified subdir's time
+    * as modified time of the requested child.
+    *
+    * @param ret contains children and modified times.
+    * @param path Name of the path to start checking dates from.
+    * @param child child of the requested path.
+    * @return modified time.
+    */
+   private long getModifiedTime(Map<String, Long> ret, String path,
+       String child) {
+     MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+     String srcPath;
+     if (path.equals(Path.SEPARATOR)) {
+       srcPath = Path.SEPARATOR + child;
+     } else {
+       srcPath = path + Path.SEPARATOR + child;
+     }
+     Long modTime = 0L;
+     try {
+       // Get mount table entry for the srcPath
+       MountTable entry = mountTable.getMountPoint(srcPath);
+       // if srcPath is not in mount table but its subdirs are in mount
+       // table we will display latest modified subdir date/time.
+       if (entry == null) {
+         List<MountTable> entries = mountTable.getMounts(srcPath);
+         for (MountTable eachEntry : entries) {
+           // Get the latest date
+           if (ret.get(child) == null ||
+               ret.get(child) < eachEntry.getDateModified()) {
+             modTime = eachEntry.getDateModified();
+           }
+         }
+       } else {
+         modTime = entry.getDateModified();
+       }
+     } catch (IOException e) {
+       LOG.error("Cannot get mount point", e);
+     }
+     return modTime;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/da182275/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 7a6816e,fe54993..5f812a5
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@@ -2082,23 -1178,21 +1178,26 @@@ public class RouterRpcServer extends Ab
    }
  
    @Deprecated
-   @Override
+   @Override // ClientProtocol
    public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
        throws IOException {
-     return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
-         OpenFilesIterator.FILTER_PATH_DEFAULT);
+     return clientProto.listOpenFiles(prevId);
    }
  
-   @Override
+   @Override // ClientProtocol
    public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
        EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
-     checkOperation(OperationCategory.READ, false);
-     return null;
+     return clientProto.listOpenFiles(prevId, openFilesTypes, path);
    }
  
-   @Override
+   @Override // ClientProtocol
 +  public void msync() throws IOException {
-     throw new UnsupportedOperationException("msync not supported");
++    clientProto.msync();
++  }
++
++  @Override // ClientProtocol
+   public void satisfyStoragePolicy(String path) throws IOException {
+     clientProto.satisfyStoragePolicy(path);
    }
  
    @Override // NamenodeProtocol


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


[21/50] [abbrv] hadoop git commit: YARN-8581. [AMRMProxy] Add sub-cluster timeout in LocalityMulticastAMRMProxyPolicy. Contributed by Botong Huang.

Posted by sh...@apache.org.
YARN-8581. [AMRMProxy] Add sub-cluster timeout in LocalityMulticastAMRMProxyPolicy. Contributed by Botong Huang.


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

Branch: refs/heads/HDFS-12943
Commit: e0f6ffdbad6f43fd43ec57fb68ebf5275b8b9ba0
Parents: 8736fc3
Author: Giovanni Matteo Fumarola <gi...@apache.com>
Authored: Mon Aug 20 14:33:16 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.com>
Committed: Mon Aug 20 14:33:16 2018 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  8 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |  2 +
 .../LocalityMulticastAMRMProxyPolicy.java       | 64 ++++++++++++--
 .../utils/FederationStateStoreFacade.java       |  9 ++
 .../TestLocalityMulticastAMRMProxyPolicy.java   | 91 +++++++++++++++++---
 .../utils/FederationPoliciesTestUtil.java       |  7 +-
 6 files changed, 162 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 78e28f7..148edb9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -3209,8 +3209,14 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.federation.resolver."
           + "DefaultSubClusterResolverImpl";
 
-  public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
+  // AMRMProxy split-merge timeout for active sub-clusters. We will not route
+  // new asks to expired sub-clusters.
+  public static final String FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT =
+      FEDERATION_PREFIX + "amrmproxy.subcluster.timeout.ms";
+  public static final long DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT =
+      60000; // one minute
 
+  public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
   public static final String FEDERATION_POLICY_MANAGER = FEDERATION_PREFIX
       + "policy-manager";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 9249ed4..d63933c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -105,6 +105,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
 
     // Federation StateStore ZK implementation configs to be ignored
     configurationPropsToSkipCompare.add(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
index 1481f34..1ccd61c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -126,6 +127,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
   private SubClusterResolver resolver;
 
   private Map<SubClusterId, Resource> headroom;
+  private Map<SubClusterId, Long> lastHeartbeatTimeStamp;
+  private long subClusterTimeOut;
   private float hrAlpha;
   private FederationStateStoreFacade federationFacade;
   private AllocationBookkeeper bookkeeper;
@@ -178,6 +181,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
 
     if (headroom == null) {
       headroom = new ConcurrentHashMap<>();
+      lastHeartbeatTimeStamp = new ConcurrentHashMap<>();
     }
     hrAlpha = policy.getHeadroomAlpha();
 
@@ -185,13 +189,29 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
         policyContext.getFederationStateStoreFacade();
     this.homeSubcluster = policyContext.getHomeSubcluster();
 
+    this.subClusterTimeOut = this.federationFacade.getConf().getLong(
+        YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+        YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
+    if (this.subClusterTimeOut <= 0) {
+      LOG.info(
+          "{} configured to be {}, should be positive. Using default of {}.",
+          YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+          this.subClusterTimeOut,
+          YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT);
+      this.subClusterTimeOut =
+          YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT;
+    }
   }
 
   @Override
   public void notifyOfResponse(SubClusterId subClusterId,
       AllocateResponse response) throws YarnException {
-    // stateless policy does not care about responses except tracking headroom
-    headroom.put(subClusterId, response.getAvailableResources());
+    if (response.getAvailableResources() != null) {
+      headroom.put(subClusterId, response.getAvailableResources());
+      LOG.info("Subcluster {} updated with {} memory headroom", subClusterId,
+          response.getAvailableResources().getMemorySize());
+    }
+    lastHeartbeatTimeStamp.put(subClusterId, System.currentTimeMillis());
   }
 
   @Override
@@ -281,6 +301,15 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     // handle all non-localized requests (ANY)
     splitAnyRequests(nonLocalizedRequests, bookkeeper);
 
+    for (Map.Entry<SubClusterId, List<ResourceRequest>> entry : bookkeeper
+        .getAnswer().entrySet()) {
+      // A new-cluster here will trigger new UAM luanch, which might take a long
+      // time. We don't want too many requests stuck in this UAM before it is
+      // ready and starts heartbeating
+      if (!lastHeartbeatTimeStamp.containsKey(entry.getKey())) {
+        lastHeartbeatTimeStamp.put(entry.getKey(), System.currentTimeMillis());
+      }
+    }
     return bookkeeper.getAnswer();
   }
 
@@ -519,13 +548,10 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
       policyWeights = weights;
       totPolicyWeight = 0;
 
-      // pre-compute the set of subclusters that are both active and enabled by
-      // the policy weights, and accumulate their total weight
       for (Map.Entry<SubClusterId, Float> entry : policyWeights.entrySet()) {
         if (entry.getValue() > 0
             && activeSubclusters.containsKey(entry.getKey())) {
           activeAndEnabledSC.add(entry.getKey());
-          totPolicyWeight += entry.getValue();
         }
       }
 
@@ -535,6 +561,34 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
                 + "currently active we cannot forward the ResourceRequest(s)");
       }
 
+      Set<SubClusterId> tmpSCSet = new HashSet<>(activeAndEnabledSC);
+      for (Map.Entry<SubClusterId, Long> entry : lastHeartbeatTimeStamp
+          .entrySet()) {
+        long duration = System.currentTimeMillis() - entry.getValue();
+        if (duration > subClusterTimeOut) {
+          LOG.warn(
+              "Subcluster {} does not have a success heartbeat for {}s, "
+                  + "skip routing asks there for this request",
+              entry.getKey(), (double) duration / 1000);
+          tmpSCSet.remove(entry.getKey());
+        }
+      }
+      if (tmpSCSet.size() < 1) {
+        LOG.warn("All active and enabled subclusters have expired last "
+            + "heartbeat time. Ignore the expiry check for this request");
+      } else {
+        activeAndEnabledSC = tmpSCSet;
+      }
+
+      LOG.info("{} subcluster active, {} subclusters active and enabled",
+          activeSubclusters.size(), activeAndEnabledSC.size());
+
+      // pre-compute the set of subclusters that are both active and enabled by
+      // the policy weights, and accumulate their total weight
+      for (SubClusterId sc : activeAndEnabledSC) {
+        totPolicyWeight += policyWeights.get(sc);
+      }
+
       // pre-compute headroom-based weights for active/enabled subclusters
       for (Map.Entry<SubClusterId, Resource> r : headroom.entrySet()) {
         if (activeAndEnabledSC.contains(r.getKey())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index 1bcb0f4..5d9702f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -392,6 +392,15 @@ public final class FederationStateStoreFacade {
   }
 
   /**
+   * Get the configuration.
+   *
+   * @return configuration object
+   */
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
    * Helper method to create instances of Object using the class name defined in
    * the configuration object. The instances creates {@link RetryProxy} using
    * the specific {@link RetryPolicy}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
index f66bbb6..cf9ac53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -32,11 +32,13 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -106,6 +108,10 @@ public class TestLocalityMulticastAMRMProxyPolicy
   }
 
   private void initializePolicy() throws YarnException {
+    initializePolicy(new YarnConfiguration());
+  }
+
+  private void initializePolicy(Configuration conf) throws YarnException {
     setFederationPolicyContext(new FederationPolicyInitializationContext());
     SubClusterResolver resolver = FederationPoliciesTestUtil.initResolver();
     getFederationPolicyContext().setFederationSubclusterResolver(resolver);
@@ -116,7 +122,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     getFederationPolicyContext().setHomeSubcluster(getHomeSubCluster());
     FederationPoliciesTestUtil.initializePolicyContext(
         getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
-        getActiveSubclusters());
+        getActiveSubclusters(), conf);
   }
 
   @Test(expected = FederationPolicyInitializationException.class)
@@ -145,7 +151,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     initializePolicy();
     List<ResourceRequest> resourceRequests = createSimpleRequest();
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -205,7 +211,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     int numRR = 1000;
     List<ResourceRequest> resourceRequests = createLargeRandomList(numRR);
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     int numIterations = 1000;
     long tstart = System.currentTimeMillis();
@@ -233,7 +239,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     List<ResourceRequest> resourceRequests = createZeroSizedANYRequest();
 
     // this receives responses from sc0,sc1,sc2
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -269,7 +275,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     initializePolicy();
     List<ResourceRequest> resourceRequests = createSimpleRequest();
 
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     Map<SubClusterId, List<ResourceRequest>> response =
         ((FederationAMRMProxyPolicy) getPolicy())
@@ -292,10 +298,14 @@ public class TestLocalityMulticastAMRMProxyPolicy
     checkTotalContainerAllocation(response, 100);
   }
 
-  private void prepPolicyWithHeadroom() throws YarnException {
+  private void prepPolicyWithHeadroom(boolean setSubCluster0)
+      throws YarnException {
     AllocateResponse ar = getAllocateResponseWithTargetHeadroom(40);
-    ((FederationAMRMProxyPolicy) getPolicy())
-        .notifyOfResponse(SubClusterId.newInstance("subcluster0"), ar);
+
+    if (setSubCluster0) {
+      ((FederationAMRMProxyPolicy) getPolicy())
+          .notifyOfResponse(SubClusterId.newInstance("subcluster0"), ar);
+    }
 
     ar = getAllocateResponseWithTargetHeadroom(0);
     ((FederationAMRMProxyPolicy) getPolicy())
@@ -333,7 +343,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     FederationPoliciesTestUtil.initializePolicyContext(
         getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
-        getActiveSubclusters());
+        getActiveSubclusters(), new Configuration());
 
     List<ResourceRequest> resourceRequests = createComplexRequest();
 
@@ -669,7 +679,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     List<ResourceRequest> resourceRequests = new ArrayList<>();
 
     // Initialize the headroom map
-    prepPolicyWithHeadroom();
+    prepPolicyWithHeadroom(true);
 
     // Cancel at ANY level only
     resourceRequests.add(FederationPoliciesTestUtil.createResourceRequest(0L,
@@ -716,4 +726,65 @@ public class TestLocalityMulticastAMRMProxyPolicy
     checkExpectedAllocation(response, "subcluster5", 1, 25);
     checkTotalContainerAllocation(response, 100);
   }
+
+  @Test
+  public void testSubClusterExpiry() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+    // Configure policy to be 100% headroom based
+    getPolicyInfo().setHeadroomAlpha(1.0f);
+
+    YarnConfiguration conf = new YarnConfiguration();
+    // Set expiry to 500ms
+    conf.setLong(YarnConfiguration.FEDERATION_AMRMPROXY_SUBCLUSTER_TIMEOUT,
+        500);
+
+    initializePolicy(conf);
+    List<ResourceRequest> resourceRequests = createSimpleRequest();
+
+    // Update the response timestamp for the first time
+    prepPolicyWithHeadroom(true);
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    /*
+     * based on headroom, we expect 75 containers to got to subcluster0 (60) and
+     * subcluster2 (15) according to the advertised headroom (40 and 10), no
+     * containers for sublcuster1 as it advertise zero headroom, and 25 to
+     * subcluster5 which has unknown headroom, and so it gets 1/4th of the load
+     */
+    checkExpectedAllocation(response, "subcluster0", 1, 60);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 15);
+    checkExpectedAllocation(response, "subcluster5", 1, 25);
+    checkTotalContainerAllocation(response, 100);
+
+    Thread.sleep(800);
+
+    // Update the response timestamp for the second time, skipping sc0 and sc5
+    prepPolicyWithHeadroom(false);
+
+    response = ((FederationAMRMProxyPolicy) getPolicy())
+        .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    checkExpectedAllocation(response, "subcluster0", 1, -1);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 100);
+    checkExpectedAllocation(response, "subcluster5", 1, -1);
+    checkTotalContainerAllocation(response, 100);
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0f6ffdb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index 24399cb..9cc6d06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -117,7 +117,7 @@ public final class FederationPoliciesTestUtil {
   public static void initializePolicyContext(
       FederationPolicyInitializationContext fpc, ConfigurableFederationPolicy
       policy, WeightedPolicyInfo policyInfo,
-      Map<SubClusterId, SubClusterInfo> activeSubclusters)
+      Map<SubClusterId, SubClusterInfo> activeSubclusters, Configuration conf)
       throws YarnException {
     ByteBuffer buf = policyInfo.toByteBuffer();
     fpc.setSubClusterPolicyConfiguration(SubClusterPolicyConfiguration
@@ -133,7 +133,7 @@ public final class FederationPoliciesTestUtil {
         .newInstance(new ArrayList<SubClusterInfo>(activeSubclusters.values()));
 
     when(fss.getSubClusters(any())).thenReturn(response);
-    facade.reinitialize(fss, new Configuration());
+    facade.reinitialize(fss, conf);
     fpc.setFederationStateStoreFacade(facade);
     policy.reinitialize(fpc);
   }
@@ -155,7 +155,8 @@ public final class FederationPoliciesTestUtil {
     FederationPolicyInitializationContext context =
         new FederationPolicyInitializationContext(null, initResolver(),
             initFacade(), SubClusterId.newInstance(subclusterId));
-    initializePolicyContext(context, policy, policyInfo, activeSubclusters);
+    initializePolicyContext(context, policy, policyInfo, activeSubclusters,
+        new Configuration());
   }
 
   /**


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


[40/50] [abbrv] hadoop git commit: HDDS-317. Use new StorageSize API for reading ozone.scm.container.size.gb. Contributed by Junjie Chen.

Posted by sh...@apache.org.
HDDS-317. Use new StorageSize API for reading ozone.scm.container.size.gb. Contributed by Junjie Chen.


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

Branch: refs/heads/HDFS-12943
Commit: 55b69310590723c5665f3e04589ef5e721d8261d
Parents: 96c4575
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 13:54:38 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 13:56:30 2018 +0200

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java    |  6 +++---
 hadoop-hdds/common/src/main/resources/ozone-default.xml   |  7 +++----
 .../hadoop/ozone/container/common/impl/ContainerData.java |  2 +-
 .../ozone/container/keyvalue/KeyValueContainerData.java   | 10 +++++++---
 .../hadoop/ozone/container/keyvalue/KeyValueHandler.java  |  7 ++++---
 .../apache/hadoop/hdds/scm/block/BlockManagerImpl.java    |  9 +++++----
 .../hadoop/hdds/scm/container/ContainerMapping.java       |  8 ++++----
 .../hadoop/hdds/scm/container/ContainerStateManager.java  |  9 +++++----
 .../hadoop/hdds/scm/pipelines/PipelineSelector.java       |  9 +++++----
 .../scm/container/TestCloseContainerEventHandler.java     |  8 ++++----
 .../hdds/scm/container/closer/TestContainerCloser.java    |  7 ++++---
 .../main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java  |  8 +++++---
 .../commandhandler/TestCloseContainerHandler.java         |  4 ++--
 .../commandhandler/TestReplicateContainerHandler.java     |  5 ++---
 14 files changed, 54 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index e337d2f..2834883 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -215,9 +215,9 @@ public final class ScmConfigKeys {
       "ozone.scm.db.cache.size.mb";
   public static final int OZONE_SCM_DB_CACHE_SIZE_DEFAULT = 128;
 
-  public static final String OZONE_SCM_CONTAINER_SIZE_GB =
-      "ozone.scm.container.size.gb";
-  public static final int OZONE_SCM_CONTAINER_SIZE_DEFAULT = 5;
+  public static final String OZONE_SCM_CONTAINER_SIZE =
+      "ozone.scm.container.size";
+  public static final String OZONE_SCM_CONTAINER_SIZE_DEFAULT = "5GB";
 
   public static final String OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY =
       "ozone.scm.container.placement.impl";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 568e38d..37a845e 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -611,12 +611,11 @@
     </description>
   </property>
   <property>
-    <name>ozone.scm.container.size.gb</name>
-    <value>5</value>
+    <name>ozone.scm.container.size</name>
+    <value>5GB</value>
     <tag>OZONE, PERFORMANCE, MANAGEMENT</tag>
     <description>
-      Default container size used by Ozone. This value is specified
-      in GB.
+      Default container size used by Ozone.
       There are two considerations while picking this number. The speed at which
       a container can be replicated, determined by the network speed and the
       metadata that each container generates. So selecting a large number

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 47894dc..afd1407 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -111,7 +111,7 @@ public abstract class ContainerData {
    * @param type - ContainerType
    * @param containerId - ContainerId
    * @param layOutVersion - Container layOutVersion
-   * @param size - Container maximum size
+   * @param size - Container maximum size in GB
    */
   protected ContainerData(ContainerType type, long containerId,
     int layOutVersion, int size) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 1d37437..e4cb5f3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.container.keyvalue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import java.util.Collections;
+
+import org.apache.hadoop.conf.StorageSize;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
@@ -82,7 +84,7 @@ public class KeyValueContainerData extends ContainerData {
   /**
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
-   * @param size - maximum size of the container
+   * @param size - maximum size in GB of the container
    */
   public KeyValueContainerData(long id, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, size);
@@ -94,7 +96,7 @@ public class KeyValueContainerData extends ContainerData {
    * Constructs KeyValueContainerData object.
    * @param id - ContainerId
    * @param layOutVersion
-   * @param size - maximum size of the container
+   * @param size - maximum size in GB of the container
    */
   public KeyValueContainerData(long id, int layOutVersion, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
@@ -266,9 +268,11 @@ public class KeyValueContainerData extends ContainerData {
   public static KeyValueContainerData getFromProtoBuf(
       ContainerProtos.ContainerData protoData) throws IOException {
     // TODO: Add containerMaxSize to ContainerProtos.ContainerData
+    StorageSize storageSize = StorageSize.parse(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
     KeyValueContainerData data = new KeyValueContainerData(
         protoData.getContainerID(),
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+        (int)storageSize.getUnit().toBytes(storageSize.getValue()));
     for (int x = 0; x < protoData.getMetadataCount(); x++) {
       data.addMetadata(protoData.getMetadata(x).getKey(),
           protoData.getMetadata(x).getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 29c359e..8409561 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -149,9 +150,9 @@ public class KeyValueHandler extends Handler {
     volumeChoosingPolicy = ReflectionUtils.newInstance(conf.getClass(
         HDDS_DATANODE_VOLUME_CHOOSING_POLICY, RoundRobinVolumeChoosingPolicy
             .class, VolumeChoosingPolicy.class), conf);
-    maxContainerSizeGB = config.getInt(ScmConfigKeys
-            .OZONE_SCM_CONTAINER_SIZE_GB, ScmConfigKeys
-        .OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    maxContainerSizeGB = (int)config.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+            ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.GB);
     // this handler lock is used for synchronizing createContainer Requests,
     // so using a fair lock here.
     handlerLock = new AutoCloseableLock(new ReentrantLock(true));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index f3a111f..ca2a6a0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.block;
 
 import java.util.UUID;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.Mapping;
 import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
@@ -30,7 +31,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
@@ -98,9 +98,10 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     this.nodeManager = nodeManager;
     this.containerManager = containerManager;
 
-    this.containerSize = OzoneConsts.GB * conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize = (long)conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
 
     this.containerProvisionBatchSize =
         conf.getInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index 4076dad..ba95764 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Longs;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
@@ -66,7 +67,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
     .FAILED_TO_CHANGE_CONTAINER_STATE;
 import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
@@ -129,9 +130,8 @@ public class ContainerMapping implements Mapping {
 
     this.lock = new ReentrantLock();
 
-    // To be replaced with code getStorageSize once it is committed.
-    size = conf.getLong(OZONE_SCM_CONTAINER_SIZE_GB,
-        OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
+    size = (long)conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     this.containerStateManager =
         new ContainerStateManager(conf, this);
     LOG.trace("Container State Manager created.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index 5df7dc7..5eb8195 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.container;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
@@ -35,7 +36,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.statemachine
     .InvalidStateTransitionException;
 import org.apache.hadoop.ozone.common.statemachine.StateMachine;
@@ -148,9 +148,10 @@ public class ContainerStateManager implements Closeable {
         finalStates);
     initializeStateMachine();
 
-    this.containerSize = OzoneConsts.GB * configuration.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize =(long)configuration.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
 
     lastUsedMap = new ConcurrentHashMap<>();
     containerCount = new AtomicLong(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index 028d14b..5343bce 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.pipelines;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
@@ -38,7 +39,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.statemachine
     .InvalidStateTransitionException;
 import org.apache.hadoop.ozone.common.statemachine.StateMachine;
@@ -94,9 +94,10 @@ public class PipelineSelector {
     this.conf = conf;
     this.eventPublisher = eventPublisher;
     this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
-    this.containerSize = OzoneConsts.GB * this.conf.getInt(
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
-        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    this.containerSize = (long)this.conf.getStorageSize(
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
+        ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.BYTES);
     node2PipelineMap = new Node2PipelineMap();
     this.standaloneManager =
         new StandaloneManagerImpl(this.nodeManager, placementPolicy,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
index 543cad3..4790c82 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.container;
 
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -39,7 +40,7 @@ import java.io.IOException;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
 
@@ -58,9 +59,8 @@ public class TestCloseContainerEventHandler {
   @BeforeClass
   public static void setUp() throws Exception {
     configuration = SCMTestUtils.getConf();
-    size = configuration
-        .getLong(OZONE_SCM_CONTAINER_SIZE_GB, OZONE_SCM_CONTAINER_SIZE_DEFAULT)
-        * 1024 * 1024 * 1024;
+    size = (long)configuration.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     testDir = GenericTestUtils
         .getTestDir(TestCloseContainerEventHandler.class.getSimpleName());
     configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
index 0c0f25d..608bb92 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdds.scm.container.closer;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.container.ContainerMapping;
@@ -50,7 +51,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent
     .CREATE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent
@@ -71,8 +72,8 @@ public class TestContainerCloser {
   @BeforeClass
   public static void setUp() throws Exception {
     configuration = SCMTestUtils.getConf();
-    size = configuration.getLong(OZONE_SCM_CONTAINER_SIZE_GB,
-        OZONE_SCM_CONTAINER_SIZE_DEFAULT) * 1024 * 1024 * 1024;
+    size = (long)configuration.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
+        OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
     configuration.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL,
         1, TimeUnit.SECONDS);
     testDir = GenericTestUtils

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
index 8d71d00..f54322c 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
@@ -23,6 +23,7 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
 import org.apache.hadoop.hdds.scm.cli.container.ContainerCommandHandler;
 import org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler;
@@ -49,7 +50,7 @@ import java.util.Arrays;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
 import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
 import static org.apache.hadoop.hdds.scm.cli.ResultCode.SUCCESS;
@@ -107,8 +108,9 @@ public class SCMCLI extends OzoneBaseCLI {
         StorageContainerLocationProtocolPB.class);
     InetSocketAddress scmAddress =
         getScmAddressForClients(ozoneConf);
-    int containerSizeGB = ozoneConf.getInt(OZONE_SCM_CONTAINER_SIZE_GB,
-        OZONE_SCM_CONTAINER_SIZE_DEFAULT);
+    int containerSizeGB = (int)ozoneConf.getStorageSize(
+        OZONE_SCM_CONTAINER_SIZE, OZONE_SCM_CONTAINER_SIZE_DEFAULT,
+        StorageUnit.GB);
     ContainerOperationClient.setContainerSizeB(containerSizeGB*OzoneConsts.GB);
 
     RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
index 3d39dbb..84b7b76 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -52,7 +52,7 @@ public class TestCloseContainerHandler {
 
     //setup a cluster (1G free space is enough for a unit test)
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OZONE_SCM_CONTAINER_SIZE_GB, "1");
+    conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
     MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55b69310/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
index a5b101f..9e08212 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestReplicateContainerHandler.java
@@ -23,14 +23,13 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
 import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_GB;
+    .OZONE_SCM_CONTAINER_SIZE;
 import org.junit.Test;
 
 /**
@@ -47,7 +46,7 @@ public class TestReplicateContainerHandler {
         .captureLogs(ReplicateContainerCommandHandler.LOG);
 
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OZONE_SCM_CONTAINER_SIZE_GB, "1");
+    conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB");
     MiniOzoneCluster cluster =
         MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();


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


[02/50] [abbrv] hadoop git commit: YARN-8612. Fix NM Collector Service Port issue in YarnConfiguration. Contributed by Prabha Manepalli.

Posted by sh...@apache.org.
YARN-8612. Fix NM Collector Service Port issue in YarnConfiguration. Contributed by Prabha Manepalli.


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

Branch: refs/heads/HDFS-12943
Commit: 1697a0230696e1ed6d9c19471463b44a6d791dfa
Parents: edeb2a3
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Aug 17 11:11:56 2018 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Aug 17 11:12:10 2018 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1697a023/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 84bfb55..78e28f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1216,7 +1216,7 @@ public class YarnConfiguration extends Configuration {
       NM_PREFIX + "collector-service.address";
   public static final int DEFAULT_NM_COLLECTOR_SERVICE_PORT = 8048;
   public static final String DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS =
-      "0.0.0.0:" + DEFAULT_NM_LOCALIZER_PORT;
+      "0.0.0.0:" + DEFAULT_NM_COLLECTOR_SERVICE_PORT;
 
   /** Interval in between cache cleanups.*/
   public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =


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


[30/50] [abbrv] hadoop git commit: HDFS-13821. RBF: Add dfs.federation.router.mount-table.cache.enable so that users can disable cache. Contributed by Fei Hui.

Posted by sh...@apache.org.
HDFS-13821. RBF: Add dfs.federation.router.mount-table.cache.enable so that users can disable cache. Contributed by Fei Hui.


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

Branch: refs/heads/HDFS-12943
Commit: 81847392badcd58d934333e7c3b5bf14b4fa1f3f
Parents: e557c6b
Author: Yiqun Lin <yq...@apache.org>
Authored: Wed Aug 22 11:43:40 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Wed Aug 22 11:43:40 2018 +0800

----------------------------------------------------------------------
 .../federation/resolver/MountTableResolver.java | 37 ++++++++++++++------
 .../server/federation/router/RBFConfigKeys.java |  4 +++
 .../src/main/resources/hdfs-rbf-default.xml     |  9 +++++
 .../resolver/TestMountTableResolver.java        | 31 ++++++++++++++++
 4 files changed, 71 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
index c264de3..d45441f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DeprecatedKeys.
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE;
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT;
 import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.isParentEntry;
 
 import java.io.IOException;
@@ -124,12 +126,19 @@ public class MountTableResolver
       this.stateStore = null;
     }
 
-    int maxCacheSize = conf.getInt(
-        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE,
-        FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT);
-    this.locationCache = CacheBuilder.newBuilder()
-        .maximumSize(maxCacheSize)
-        .build();
+    boolean mountTableCacheEnable = conf.getBoolean(
+        FEDERATION_MOUNT_TABLE_CACHE_ENABLE,
+        FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT);
+    if (mountTableCacheEnable) {
+      int maxCacheSize = conf.getInt(
+          FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE,
+          FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT);
+      this.locationCache = CacheBuilder.newBuilder()
+          .maximumSize(maxCacheSize)
+          .build();
+    } else {
+      this.locationCache = null;
+    }
 
     registerCacheExternal();
     initDefaultNameService(conf);
@@ -239,7 +248,7 @@ public class MountTableResolver
    */
   private void invalidateLocationCache(final String path) {
     LOG.debug("Invalidating {} from {}", path, locationCache);
-    if (locationCache.size() == 0) {
+    if (locationCache == null || locationCache.size() == 0) {
       return;
     }
 
@@ -359,7 +368,9 @@ public class MountTableResolver
     LOG.info("Clearing all mount location caches");
     writeLock.lock();
     try {
-      this.locationCache.invalidateAll();
+      if (this.locationCache != null) {
+        this.locationCache.invalidateAll();
+      }
       this.tree.clear();
     } finally {
       writeLock.unlock();
@@ -372,6 +383,9 @@ public class MountTableResolver
     verifyMountTable();
     readLock.lock();
     try {
+      if (this.locationCache == null) {
+        return lookupLocation(path);
+      }
       Callable<? extends PathLocation> meh = new Callable<PathLocation>() {
         @Override
         public PathLocation call() throws Exception {
@@ -603,7 +617,10 @@ public class MountTableResolver
    * Get the size of the cache.
    * @return Size of the cache.
    */
-  protected long getCacheSize() {
-    return this.locationCache.size();
+  protected long getCacheSize() throws IOException{
+    if (this.locationCache != null) {
+      return this.locationCache.size();
+    }
+    throw new IOException("localCache is null");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index 363db20..87df5d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -186,6 +186,10 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
       FEDERATION_ROUTER_PREFIX + "mount-table.max-cache-size";
   /** Remove cache entries if we have more than 10k. */
   public static final int FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE_DEFAULT = 10000;
+  public static final String FEDERATION_MOUNT_TABLE_CACHE_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "mount-table.cache.enable";
+  public static final boolean FEDERATION_MOUNT_TABLE_CACHE_ENABLE_DEFAULT =
+      true;
 
   // HDFS Router-based federation admin
   public static final String DFS_ROUTER_ADMIN_HANDLER_COUNT_KEY =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index 8806cb2..8be5b8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -394,6 +394,15 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.mount-table.cache.enable</name>
+    <value>true</value>
+    <description>
+      Set to true to enable mount table cache (Path to Remote Location cache).
+      Disabling the cache is recommended when a large amount of unique paths are queried.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.quota.enable</name>
     <value>false</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81847392/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
index cb3b472..b19a973 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestMountTableResolver.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
+import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_CACHE_ENABLE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.FEDERATION_MOUNT_TABLE_MAX_CACHE_SIZE;
 import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
 import static org.junit.Assert.assertEquals;
@@ -37,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -473,6 +475,35 @@ public class TestMountTableResolver {
   }
 
   @Test
+  public void testDisableLocalCache() throws IOException {
+    Configuration conf = new Configuration();
+    // Disable mount table cache
+    conf.setBoolean(FEDERATION_MOUNT_TABLE_CACHE_ENABLE, false);
+    conf.setStrings(DFS_ROUTER_DEFAULT_NAMESERVICE, "0");
+    MountTableResolver tmpMountTable = new MountTableResolver(conf);
+
+    // Root mount point
+    Map<String, String> map = getMountTableEntry("1", "/");
+    tmpMountTable.addEntry(MountTable.newInstance("/", map));
+
+    // /tmp
+    map = getMountTableEntry("2", "/tmp");
+    tmpMountTable.addEntry(MountTable.newInstance("/tmp", map));
+
+    // Check localCache is null
+    try {
+      tmpMountTable.getCacheSize();
+      fail("getCacheSize call should fail.");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("localCache is null", e);
+    }
+
+    // Check resolve path without cache
+    assertEquals("2->/tmp/tesfile1.txt",
+        tmpMountTable.getDestinationForPath("/tmp/tesfile1.txt").toString());
+  }
+
+  @Test
   public void testCacheCleaning() throws Exception {
     for (int i = 0; i < 1000; i++) {
       String filename = String.format("/user/a/file-%04d.txt", i);


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


[17/50] [abbrv] hadoop git commit: HDDS-353. Multiple delete Blocks tests are failing consistently. Contributed by Lokesh Jain.

Posted by sh...@apache.org.
HDDS-353. Multiple delete Blocks tests are failing consistently. Contributed by Lokesh Jain.


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

Branch: refs/heads/HDFS-12943
Commit: e3d73bbc24eea8d539811ba07c695b1d0c139b61
Parents: 6425ed2
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Mon Aug 20 13:37:58 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Mon Aug 20 13:37:58 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/HddsServerUtil.java   |  6 +++---
 .../report/CommandStatusReportPublisher.java     |  2 +-
 .../common/report/ContainerReportPublisher.java  |  2 +-
 .../common/report/NodeReportPublisher.java       |  2 +-
 .../statemachine/DatanodeStateMachine.java       |  3 +--
 .../statemachine/EndpointStateMachine.java       |  9 +++++----
 .../hdds/scm/container/ContainerMapping.java     |  7 ++++---
 .../scm/container/closer/ContainerCloser.java    |  9 ++++-----
 .../ozone/TestStorageContainerManager.java       | 19 ++++++++++++++-----
 .../commandhandler/TestBlockDeletion.java        | 10 +++++++---
 .../apache/hadoop/ozone/web/client/TestKeys.java | 10 +++++++++-
 11 files changed, 50 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
index a8b919d..580d027 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/hdds/scm/HddsServerUtil.java
@@ -180,11 +180,11 @@ public final class HddsServerUtil {
    * SCM.
    *
    * @param conf - Ozone Config
-   * @return - HB interval in seconds.
+   * @return - HB interval in milli seconds.
    */
   public static long getScmHeartbeatInterval(Configuration conf) {
     return conf.getTimeDuration(HDDS_HEARTBEAT_INTERVAL,
-        HDDS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+        HDDS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
   }
 
   /**
@@ -202,7 +202,7 @@ public final class HddsServerUtil {
 
     long heartbeatThreadFrequencyMs = getScmheartbeatCheckerInterval(conf);
 
-    long heartbeatIntervalMs = getScmHeartbeatInterval(conf) * 1000;
+    long heartbeatIntervalMs = getScmHeartbeatInterval(conf);
 
 
     // Make sure that StaleNodeInterval is configured way above the frequency

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/CommandStatusReportPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/CommandStatusReportPublisher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/CommandStatusReportPublisher.java
index 3898d15..4cf6321 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/CommandStatusReportPublisher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/CommandStatusReportPublisher.java
@@ -58,7 +58,7 @@ public class CommandStatusReportPublisher extends
           getConf());
 
       Preconditions.checkState(
-          heartbeatFrequency < cmdStatusReportInterval,
+          heartbeatFrequency <= cmdStatusReportInterval,
           HDDS_COMMAND_STATUS_REPORT_INTERVAL +
               " cannot be configured lower than heartbeat frequency.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/ContainerReportPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/ContainerReportPublisher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/ContainerReportPublisher.java
index 3e73bb4..ccb9a9a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/ContainerReportPublisher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/ContainerReportPublisher.java
@@ -64,7 +64,7 @@ public class ContainerReportPublisher extends
           getConf());
 
       Preconditions.checkState(
-          heartbeatFrequency < containerReportInterval,
+          heartbeatFrequency <= containerReportInterval,
           HDDS_CONTAINER_REPORT_INTERVAL +
               " cannot be configured lower than heartbeat frequency.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/NodeReportPublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/NodeReportPublisher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/NodeReportPublisher.java
index b98cba4..6ac99dd 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/NodeReportPublisher.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/report/NodeReportPublisher.java
@@ -52,7 +52,7 @@ public class NodeReportPublisher extends ReportPublisher<NodeReportProto> {
           getConf());
 
       Preconditions.checkState(
-          heartbeatFrequency < nodeReportInterval,
+          heartbeatFrequency <= nodeReportInterval,
           HDDS_NODE_REPORT_INTERVAL +
               " cannot be configured lower than heartbeat frequency.");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index 1ac42dd..cf6f1ca 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -86,8 +86,7 @@ public class DatanodeStateMachine implements Closeable {
             .setNameFormat("Datanode State Machine Thread - %d").build());
     connectionManager = new SCMConnectionManager(conf);
     context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
-    heartbeatFrequency = TimeUnit.SECONDS.toMillis(
-        getScmHeartbeatInterval(conf));
+    heartbeatFrequency = getScmHeartbeatInterval(conf);
     container = new OzoneContainer(this.datanodeDetails,
         new OzoneConfiguration(conf), context);
     nextHB = new AtomicLong(Time.monotonicNow());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
index fb32a05..5f78a33 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/EndpointStateMachine.java
@@ -27,6 +27,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.time.ZonedDateTime;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -203,11 +204,11 @@ public class EndpointStateMachine
     this.incMissed();
     if (this.getMissedCount() % getLogWarnInterval(conf) ==
         0) {
-      LOG.error("Unable to communicate to SCM server at {}. We have not been " +
-              "able to communicate to this SCM server for past {} seconds.",
+      LOG.error(
+          "Unable to communicate to SCM server at {} for past {} seconds.",
           this.getAddress().getHostString() + ":" + this.getAddress().getPort(),
-          this.getMissedCount() * getScmHeartbeatInterval(
-              this.conf), ex);
+          TimeUnit.MILLISECONDS.toSeconds(
+              this.getMissedCount() * getScmHeartbeatInterval(this.conf)), ex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index e8392f5..11863f2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -217,9 +217,10 @@ public class ContainerMapping implements Mapping {
         // For close containers create pipeline from datanodes with replicas
         Set<DatanodeDetails> dnWithReplicas = containerStateManager
             .getContainerReplicas(contInfo.containerID());
-        pipeline = new Pipeline(dnWithReplicas.iterator().next().getHostName(),
-            contInfo.getState(), ReplicationType.STAND_ALONE,
-            contInfo.getReplicationFactor(), PipelineID.randomId());
+        pipeline =
+            new Pipeline(dnWithReplicas.iterator().next().getUuidString(),
+                contInfo.getState(), ReplicationType.STAND_ALONE,
+                contInfo.getReplicationFactor(), PipelineID.randomId());
         dnWithReplicas.forEach(pipeline::addMember);
       }
       return new ContainerWithPipeline(contInfo, pipeline);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java
index 26b1548..7e04928 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/closer/ContainerCloser.java
@@ -77,7 +77,7 @@ public class ContainerCloser {
     this.isRunning = new AtomicBoolean(false);
     this.reportInterval = this.configuration.getTimeDuration(
         HDDS_CONTAINER_REPORT_INTERVAL,
-        HDDS_CONTAINER_REPORT_INTERVAL_DEFAULT, TimeUnit.SECONDS);
+        HDDS_CONTAINER_REPORT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
     Preconditions.checkState(this.reportInterval > 0,
         "report interval has to be greater than 0");
   }
@@ -100,7 +100,7 @@ public class ContainerCloser {
     if (commandIssued.containsKey(info.getContainerID())) {
       // We check if we issued a close command in last 3 * reportInterval secs.
       long commandQueueTime = commandIssued.get(info.getContainerID());
-      long currentTime = TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow());
+      long currentTime = Time.monotonicNow();
       if (currentTime > commandQueueTime + (MULTIPLIER * reportInterval)) {
         commandIssued.remove(info.getContainerID());
         mapCount.decrementAndGet();
@@ -137,8 +137,7 @@ public class ContainerCloser {
               PipelineID.getFromProtobuf(info.getPipelineID())));
     }
     if (!commandIssued.containsKey(info.getContainerID())) {
-      commandIssued.put(info.getContainerID(),
-          TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow()));
+      commandIssued.put(info.getContainerID(), Time.monotonicNow());
       mapCount.incrementAndGet();
     }
     // run the hash map cleaner thread if needed, non-blocking call.
@@ -156,7 +155,7 @@ public class ContainerCloser {
           for (Map.Entry<Long, Long> entry : commandIssued.entrySet()) {
             long commandQueueTime = entry.getValue();
             if (commandQueueTime + (MULTIPLIER * reportInterval) >
-                TimeUnit.MILLISECONDS.toSeconds(Time.monotonicNow())) {
+                Time.monotonicNow()) {
 
               // It is possible for this remove to fail due to race conditions.
               // No big deal we will cleanup next time.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index c5d8747..8762c0e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -67,7 +67,8 @@ import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 import org.apache.hadoop.test.GenericTestUtils;
 
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
+import static org.apache.hadoop.hdds
+    .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
 import static org.junit.Assert.fail;
 
 /**
@@ -188,7 +189,8 @@ public class TestStorageContainerManager {
   public void testBlockDeletionTransactions() throws Exception {
     int numKeys = 5;
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 5, TimeUnit.SECONDS);
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 100,
+        TimeUnit.MILLISECONDS);
     conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
         3000,
         TimeUnit.MILLISECONDS);
@@ -200,7 +202,9 @@ public class TestStorageContainerManager {
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
         numKeys);
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
+    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
+        .setHbInterval(100)
+        .build();
     cluster.waitForClusterToBeReady();
 
     try {
@@ -212,6 +216,8 @@ public class TestStorageContainerManager {
       TestStorageContainerManagerHelper helper =
           new TestStorageContainerManagerHelper(cluster, conf);
       Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
+      // Wait for container report
+      Thread.sleep(1000);
       for (OmKeyInfo keyInfo : keyLocations.values()) {
         OzoneTestUtils.closeContainers(keyInfo.getKeyLocationVersions(),
             cluster.getStorageContainerManager());
@@ -271,14 +277,15 @@ public class TestStorageContainerManager {
   public void testBlockDeletingThrottling() throws Exception {
     int numKeys = 15;
     OzoneConfiguration conf = new OzoneConfiguration();
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS);
     conf.setInt(ScmConfigKeys.OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 5);
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
-        1000, TimeUnit.MILLISECONDS);
+        100, TimeUnit.MILLISECONDS);
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
         numKeys);
 
     MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
-        .setHbInterval(5000)
+        .setHbInterval(1000)
         .setHbProcessorInterval(3000)
         .build();
     cluster.waitForClusterToBeReady();
@@ -298,6 +305,8 @@ public class TestStorageContainerManager {
     TestStorageContainerManagerHelper helper =
         new TestStorageContainerManagerHelper(cluster, conf);
     Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
+    // Wait for container report
+    Thread.sleep(5000);
     for (OmKeyInfo keyInfo : keyLocations.values()) {
       OzoneTestUtils.closeContainers(keyInfo.getKeyLocationVersions(),
           cluster.getStorageContainerManager());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 45659bd..2524de6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -60,7 +60,8 @@ import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
+import static org.apache.hadoop.hdds
+    .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 
@@ -89,10 +90,13 @@ public class TestBlockDeletion {
     conf.setQuietMode(false);
     conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100,
         TimeUnit.MILLISECONDS);
-    conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 200,
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200,
         TimeUnit.MILLISECONDS);
 
-    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(1)
+        .setHbInterval(200)
+        .build();
     cluster.waitForClusterToBeReady();
     store = OzoneClientFactory.getRpcClient(conf).getObjectStore();
     dnContainerSet = cluster.getHddsDatanodes().get(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d73bbc/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 2d6abe0..fd8f420 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -68,6 +68,7 @@ import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -91,6 +92,8 @@ import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import static org.apache.hadoop.hdds
+    .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -137,13 +140,17 @@ public class TestKeys {
     // Set short block deleting service interval to speed up deletions.
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
         1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS);
     conf.setBoolean(ScmConfigKeys.DFS_CONTAINER_GRPC_ENABLED_KEY,
         shouldUseGrpc);
 
     path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
-    ozoneCluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
+    ozoneCluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(1)
+        .setHbInterval(1000)
+        .build();
     ozoneCluster.waitForClusterToBeReady();
     client = new RpcClient(conf);
     currentTime = Time.now();
@@ -663,6 +670,7 @@ public class TestKeys {
   }
 
   @Test
+  @Ignore("Until delete background service is fixed.")
   public void testDeleteKey() throws Exception {
     OzoneManager ozoneManager = ozoneCluster.getOzoneManager();
     // To avoid interference from other test cases,


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


[28/50] [abbrv] hadoop git commit: YARN-8298. Added express upgrade for YARN service. Contributed by Chandni Singh

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java
new file mode 100644
index 0000000..98e7474
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/utils/TestServiceApiUtil.java
@@ -0,0 +1,743 @@
+/*
+ * 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.service.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.ServiceTestUtils;
+import org.apache.hadoop.yarn.service.api.records.Artifact;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.KerberosPrincipal;
+import org.apache.hadoop.yarn.service.api.records.PlacementConstraint;
+import org.apache.hadoop.yarn.service.api.records.PlacementPolicy;
+import org.apache.hadoop.yarn.service.api.records.PlacementScope;
+import org.apache.hadoop.yarn.service.api.records.PlacementType;
+import org.apache.hadoop.yarn.service.api.records.Resource;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.yarn.service.conf.RestApiConstants.DEFAULT_UNLIMITED_LIFETIME;
+import static org.apache.hadoop.yarn.service.exceptions.RestApiErrorMessages.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for ServiceApiUtil helper methods.
+ */
+public class TestServiceApiUtil extends ServiceTestUtils {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestServiceApiUtil.class);
+  private static final String EXCEPTION_PREFIX = "Should have thrown " +
+      "exception: ";
+  private static final String NO_EXCEPTION_PREFIX = "Should not have thrown " +
+      "exception: ";
+
+  private static final String LEN_64_STR =
+      "abcdefghijklmnopqrstuvwxyz0123456789abcdefghijklmnopqrstuvwxyz01";
+
+  private static final YarnConfiguration CONF_DEFAULT_DNS = new
+      YarnConfiguration();
+  private static final YarnConfiguration CONF_DNS_ENABLED = new
+      YarnConfiguration();
+
+  @BeforeClass
+  public static void init() {
+    CONF_DNS_ENABLED.setBoolean(RegistryConstants.KEY_DNS_ENABLED, true);
+  }
+
+  @Test(timeout = 90000)
+  public void testResourceValidation() throws Exception {
+    assertEquals(RegistryConstants.MAX_FQDN_LABEL_LENGTH + 1, LEN_64_STR
+        .length());
+
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    Service app = new Service();
+
+    // no name
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
+    }
+
+    app.setName("test");
+    // no version
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + " service with no version");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(ERROR_APPLICATION_VERSION_INVALID,
+          app.getName()), e.getMessage());
+    }
+
+    app.setVersion("v1");
+    // bad format name
+    String[] badNames = {"4finance", "Finance", "finance@home", LEN_64_STR};
+    for (String badName : badNames) {
+      app.setName(badName);
+      try {
+        ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+        Assert.fail(EXCEPTION_PREFIX + "service with bad name " + badName);
+      } catch (IllegalArgumentException e) {
+
+      }
+    }
+
+    // launch command not specified
+    app.setName(LEN_64_STR);
+    Component comp = new Component().name("comp1");
+    app.addComponent(comp);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
+      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
+    } catch (IllegalArgumentException e) {
+      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
+          e.getMessage());
+    }
+
+    // launch command not specified
+    app.setName(LEN_64_STR.substring(0, RegistryConstants
+        .MAX_FQDN_LABEL_LENGTH));
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no launch command");
+    } catch (IllegalArgumentException e) {
+      assertEquals(RestApiErrorMessages.ERROR_ABSENT_LAUNCH_COMMAND,
+          e.getMessage());
+    }
+
+    // memory not specified
+    comp.setLaunchCommand("sleep 1");
+    Resource res = new Resource();
+    app.setResource(res);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no memory");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID,
+          comp.getName()), e.getMessage());
+    }
+
+    // invalid no of cpus
+    res.setMemory("100mb");
+    res.setCpus(-2);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(
+          EXCEPTION_PREFIX + "service with invalid no of cpus");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE,
+          comp.getName()), e.getMessage());
+    }
+
+    // number of containers not specified
+    res.setCpus(2);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no container count");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains(ERROR_CONTAINERS_COUNT_INVALID));
+    }
+
+    // specifying profile along with cpus/memory raises exception
+    res.setProfile("hbase_finance_large");
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX
+          + "service with resource profile along with cpus/memory");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(RestApiErrorMessages
+              .ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
+          comp.getName()),
+          e.getMessage());
+    }
+
+    // currently resource profile alone is not supported.
+    // TODO: remove the next test once resource profile alone is supported.
+    res.setCpus(null);
+    res.setMemory(null);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with resource profile only");
+    } catch (IllegalArgumentException e) {
+      assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
+          e.getMessage());
+    }
+
+    // unset profile here and add cpus/memory back
+    res.setProfile(null);
+    res.setCpus(2);
+    res.setMemory("2gb");
+
+    // null number of containers
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "null number of containers");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith(ERROR_CONTAINERS_COUNT_INVALID));
+    }
+  }
+
+  @Test
+  public void testArtifacts() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    Service app = new Service();
+    app.setName("service1");
+    app.setVersion("v1");
+    Resource res = new Resource();
+    app.setResource(res);
+    res.setMemory("512M");
+
+    // no artifact id fails with default type
+    Artifact artifact = new Artifact();
+    app.setArtifact(artifact);
+    String compName = "comp1";
+    Component comp = ServiceTestUtils.createComponent(compName);
+
+    app.setComponents(Collections.singletonList(comp));
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
+    }
+
+    // no artifact id fails with SERVICE type
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
+    } catch (IllegalArgumentException e) {
+      assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+    }
+
+    // no artifact id fails with TARBALL type
+    artifact.setType(Artifact.TypeEnum.TARBALL);
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with no artifact id");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(ERROR_ARTIFACT_ID_FOR_COMP_INVALID, compName),
+          e.getMessage());
+    }
+
+    // everything valid here
+    artifact.setType(Artifact.TypeEnum.DOCKER);
+    artifact.setId("docker.io/centos:centos7");
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      LOG.error("service attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
+  }
+
+  private static Resource createValidResource() {
+    Resource res = new Resource();
+    res.setMemory("512M");
+    return res;
+  }
+
+  private static Component createValidComponent(String compName) {
+    Component comp = new Component();
+    comp.setName(compName);
+    comp.setResource(createValidResource());
+    comp.setNumberOfContainers(1L);
+    comp.setLaunchCommand("sleep 1");
+    return comp;
+  }
+
+  private static Service createValidApplication(String compName) {
+    Service app = new Service();
+    app.setName("name");
+    app.setVersion("v1");
+    app.setResource(createValidResource());
+    if (compName != null) {
+      app.addComponent(createValidComponent(compName));
+    }
+    return app;
+  }
+
+  @Test
+  public void testExternalApplication() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication(null);
+
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.setArtifact(artifact);
+    app.addComponent(ServiceTestUtils.createComponent("comp2"));
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    assertNotNull(app.getComponent("comp2"));
+  }
+
+  @Test
+  public void testDuplicateComponents() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+
+    String compName = "comp1";
+    Service app = createValidApplication(compName);
+    app.addComponent(createValidComponent(compName));
+
+    // duplicate component name fails
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with component collision");
+    } catch (IllegalArgumentException e) {
+      assertEquals("Component name collision: " + compName, e.getMessage());
+    }
+  }
+
+  @Test
+  public void testComponentNameSameAsServiceName() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = new Service();
+    app.setName("test");
+    app.setVersion("v1");
+    app.addComponent(createValidComponent("test"));
+
+    //component name same as service name
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "component name matches service name");
+    } catch (IllegalArgumentException e) {
+      assertEquals("Component name test must not be same as service name test",
+          e.getMessage());
+    }
+  }
+
+  @Test
+  public void testExternalDuplicateComponent() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication("comp1");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.getComponent("comp1").setArtifact(artifact);
+
+    // duplicate component name okay in the case of SERVICE component
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testExternalComponent() throws IOException {
+    Service ext = createValidApplication("comp1");
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs(ext);
+
+    Service app = createValidApplication("comp2");
+    Artifact artifact = new Artifact();
+    artifact.setType(Artifact.TypeEnum.SERVICE);
+    artifact.setId("id");
+    app.setArtifact(artifact);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    // artifact ID not inherited from global
+    assertNotNull(app.getComponent("comp2"));
+
+    // set SERVICE artifact id on component
+    app.getComponent("comp2").setArtifact(artifact);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    assertEquals(1, app.getComponents().size());
+    // original component replaced by external component
+    assertNotNull(app.getComponent("comp1"));
+  }
+
+  public static void verifyDependencySorting(List<Component> components,
+      Component... expectedSorting) {
+    Collection<Component> actualSorting = ServiceApiUtil.sortByDependencies(
+        components);
+    assertEquals(expectedSorting.length, actualSorting.size());
+    int i = 0;
+    for (Component component : actualSorting) {
+      assertEquals(expectedSorting[i++], component);
+    }
+  }
+
+  @Test
+  public void testDependencySorting() throws IOException {
+    Component a = ServiceTestUtils.createComponent("a");
+    Component b = ServiceTestUtils.createComponent("b");
+    Component c = ServiceTestUtils.createComponent("c");
+    Component d =
+        ServiceTestUtils.createComponent("d").dependencies(Arrays.asList("c"));
+    Component e = ServiceTestUtils.createComponent("e")
+        .dependencies(Arrays.asList("b", "d"));
+
+    verifyDependencySorting(Arrays.asList(a, b, c), a, b, c);
+    verifyDependencySorting(Arrays.asList(c, a, b), c, a, b);
+    verifyDependencySorting(Arrays.asList(a, b, c, d, e), a, b, c, d, e);
+    verifyDependencySorting(Arrays.asList(e, d, c, b, a), c, b, a, d, e);
+
+    c.setDependencies(Arrays.asList("e"));
+    try {
+      verifyDependencySorting(Arrays.asList(a, b, c, d, e));
+      Assert.fail(EXCEPTION_PREFIX + "components with dependency cycle");
+    } catch (IllegalArgumentException ex) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_DEPENDENCY_CYCLE, Arrays.asList(c, d,
+              e)), ex.getMessage());
+    }
+
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service service = createValidApplication(null);
+    service.setComponents(Arrays.asList(c, d, e));
+    try {
+      ServiceApiUtil.validateAndResolveService(service, sfs,
+          CONF_DEFAULT_DNS);
+      Assert.fail(EXCEPTION_PREFIX + "components with bad dependencies");
+    } catch (IllegalArgumentException ex) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_DEPENDENCY_INVALID, "b", "e"), ex
+          .getMessage());
+    }
+  }
+
+  @Test
+  public void testInvalidComponent() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    testComponent(sfs);
+  }
+
+  @Test
+  public void testValidateCompName() {
+    String[] invalidNames = {
+        "EXAMPLE", // UPPER case not allowed
+        "example_app" // underscore not allowed.
+    };
+    for (String name : invalidNames) {
+      try {
+        ServiceApiUtil.validateNameFormat(name, new Configuration());
+        Assert.fail();
+      } catch (IllegalArgumentException ex) {
+        ex.printStackTrace();
+      }
+    }
+  }
+
+  private static void testComponent(SliderFileSystem sfs)
+      throws IOException {
+    int maxLen = RegistryConstants.MAX_FQDN_LABEL_LENGTH;
+    assertEquals(19, Long.toString(Long.MAX_VALUE).length());
+    maxLen = maxLen - Long.toString(Long.MAX_VALUE).length();
+
+    String compName = LEN_64_STR.substring(0, maxLen + 1);
+    Service app = createValidApplication(null);
+    app.addComponent(createValidComponent(compName));
+
+    // invalid component name fails if dns is enabled
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid component name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(RestApiErrorMessages
+          .ERROR_COMPONENT_NAME_INVALID, maxLen, compName), e.getMessage());
+    }
+
+    // does not fail if dns is disabled
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DEFAULT_DNS);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    compName = LEN_64_STR.substring(0, maxLen);
+    app = createValidApplication(null);
+    app.addComponent(createValidComponent(compName));
+
+    // does not fail
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testPlacementPolicy() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = createValidApplication("comp-a");
+    Component comp = app.getComponents().get(0);
+    PlacementPolicy pp = new PlacementPolicy();
+    PlacementConstraint pc = new PlacementConstraint();
+    pc.setName("CA1");
+    pp.setConstraints(Collections.singletonList(pc));
+    comp.setPlacementPolicy(pp);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no type");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TYPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the type
+    pc.setType(PlacementType.ANTI_AFFINITY);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no scope");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_SCOPE_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set the scope
+    pc.setScope(PlacementScope.NODE);
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with no tag(s)");
+    } catch (IllegalArgumentException e) {
+      assertEquals(String.format(
+          RestApiErrorMessages.ERROR_PLACEMENT_POLICY_CONSTRAINT_TAGS_NULL,
+          "CA1 ", "comp-a"), e.getMessage());
+    }
+
+    // Set a target tag - but an invalid one
+    pc.setTargetTags(Collections.singletonList("comp-invalid"));
+
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+      Assert.fail(EXCEPTION_PREFIX + "constraint with invalid tag name");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(
+              RestApiErrorMessages.ERROR_PLACEMENT_POLICY_TAG_NAME_NOT_SAME,
+              "comp-invalid", "comp-a", "comp-a", "comp-a"),
+          e.getMessage());
+    }
+
+    // Set valid target tags now
+    pc.setTargetTags(Collections.singletonList("comp-a"));
+
+    // Finally it should succeed
+    try {
+      ServiceApiUtil.validateAndResolveService(app, sfs, CONF_DNS_ENABLED);
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testKerberosPrincipal() throws IOException {
+    SliderFileSystem sfs = ServiceTestUtils.initMockFs();
+    Service app = createValidApplication("comp-a");
+    KerberosPrincipal kp = new KerberosPrincipal();
+    kp.setKeytab("/some/path");
+    kp.setPrincipalName("user/_HOST@domain.com");
+    app.setKerberosPrincipal(kp);
+
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab URI scheme");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(RestApiErrorMessages.ERROR_KEYTAB_URI_SCHEME_INVALID,
+              kp.getKeytab()),
+          e.getMessage());
+    }
+
+    kp.setKeytab("/ blank / in / paths");
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid keytab");
+    } catch (IllegalArgumentException e) {
+      // strip out the %s at the end of the RestApiErrorMessages string constant
+      assertTrue(e.getMessage().contains(
+          RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.substring(0,
+              RestApiErrorMessages.ERROR_KEYTAB_URI_INVALID.length() - 2)));
+    }
+
+    kp.setKeytab("file:///tmp/a.keytab");
+    // now it should succeed
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testKerberosPrincipalNameFormat() throws IOException {
+    Service app = createValidApplication("comp-a");
+    KerberosPrincipal kp = new KerberosPrincipal();
+    kp.setPrincipalName("user@domain.com");
+    app.setKerberosPrincipal(kp);
+
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+      Assert.fail(EXCEPTION_PREFIX + "service with invalid principal name " +
+          "format.");
+    } catch (IllegalArgumentException e) {
+      assertEquals(
+          String.format(
+              RestApiErrorMessages.ERROR_KERBEROS_PRINCIPAL_NAME_FORMAT,
+              kp.getPrincipalName()),
+          e.getMessage());
+    }
+
+    kp.setPrincipalName("user/_HOST@domain.com");
+    try {
+      ServiceApiUtil.validateKerberosPrincipal(app.getKerberosPrincipal());
+    } catch (IllegalArgumentException e) {
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testResolveCompsDependency() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    dependencies.add("compb");
+    Component compa = createComponent("compa");
+    compa.setDependencies(dependencies);
+    Component compb = createComponent("compb");
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compb");
+    expected.add("compa");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveCompsDependencyReversed() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    dependencies.add("compa");
+    Component compa = createComponent("compa");
+    Component compb = createComponent("compb");
+    compb.setDependencies(dependencies);
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveCompsCircularDependency() {
+    Service service = createExampleApplication();
+    List<String> dependencies = new ArrayList<String>();
+    List<String> dependencies2 = new ArrayList<String>();
+    dependencies.add("compb");
+    dependencies2.add("compa");
+    Component compa = createComponent("compa");
+    compa.setDependencies(dependencies);
+    Component compb = createComponent("compb");
+    compa.setDependencies(dependencies2);
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  @Test
+  public void testResolveNoCompsDependency() {
+    Service service = createExampleApplication();
+    Component compa = createComponent("compa");
+    Component compb = createComponent("compb");
+    service.addComponent(compa);
+    service.addComponent(compb);
+    List<String> order = ServiceApiUtil.resolveCompsDependency(service);
+    List<String> expected = new ArrayList<String>();
+    expected.add("compa");
+    expected.add("compb");
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Components are not equal.", expected.get(i),
+          order.get(i));
+    }
+  }
+
+  public static Service createExampleApplication() {
+
+    Service exampleApp = new Service();
+    exampleApp.setName("example-app");
+    exampleApp.setVersion("v1");
+    return exampleApp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index 807938c..a0e4e02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.client.cli;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
@@ -100,6 +101,7 @@ public class ApplicationCLI extends YarnCLI {
   public static final String COMPONENT = "component";
   public static final String ENABLE_FAST_LAUNCH = "enableFastLaunch";
   public static final String UPGRADE_CMD = "upgrade";
+  public static final String UPGRADE_EXPRESS = "express";
   public static final String UPGRADE_INITIATE = "initiate";
   public static final String UPGRADE_AUTO_FINALIZE = "autoFinalize";
   public static final String UPGRADE_FINALIZE = "finalize";
@@ -247,6 +249,9 @@ public class ApplicationCLI extends YarnCLI {
       opts.addOption(UPGRADE_CMD, true, "Upgrades an application/long-" +
           "running service. It requires either -initiate, -instances, or " +
           "-finalize options.");
+      opts.addOption(UPGRADE_EXPRESS, true, "Works with -upgrade option to " +
+          "perform express upgrade.  It requires the upgraded application " +
+          "specification file.");
       opts.addOption(UPGRADE_INITIATE, true, "Works with -upgrade option to " +
           "initiate the application upgrade. It requires the upgraded " +
           "application specification file.");
@@ -639,9 +644,9 @@ public class ApplicationCLI extends YarnCLI {
       moveApplicationAcrossQueues(cliParser.getOptionValue(APP_ID),
           cliParser.getOptionValue(CHANGE_APPLICATION_QUEUE));
     } else if (cliParser.hasOption(UPGRADE_CMD)) {
-      if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD, UPGRADE_INITIATE,
-          UPGRADE_AUTO_FINALIZE, UPGRADE_FINALIZE, COMPONENT_INSTS, COMPONENTS,
-          APP_TYPE_CMD)) {
+      if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD, UPGRADE_EXPRESS,
+          UPGRADE_INITIATE, UPGRADE_AUTO_FINALIZE, UPGRADE_FINALIZE,
+          COMPONENT_INSTS, COMPONENTS, APP_TYPE_CMD)) {
         printUsage(title, opts);
         return exitCode;
       }
@@ -649,7 +654,14 @@ public class ApplicationCLI extends YarnCLI {
       AppAdminClient client =  AppAdminClient.createAppAdminClient(appType,
           getConf());
       String appName = cliParser.getOptionValue(UPGRADE_CMD);
-      if (cliParser.hasOption(UPGRADE_INITIATE)) {
+      if (cliParser.hasOption(UPGRADE_EXPRESS)) {
+        File file = new File(cliParser.getOptionValue(UPGRADE_EXPRESS));
+        if (!file.exists()) {
+          System.err.println(file.getAbsolutePath() + " does not exist.");
+          return exitCode;
+        }
+        return client.actionUpgradeExpress(appName, file);
+      } else if (cliParser.hasOption(UPGRADE_INITIATE)) {
         if (hasAnyOtherCLIOptions(cliParser, opts, UPGRADE_CMD,
             UPGRADE_INITIATE, UPGRADE_AUTO_FINALIZE, APP_TYPE_CMD)) {
           printUsage(title, opts);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index 526adfd..20c9603 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -2161,6 +2161,10 @@ public class TestYarnCLI {
     pw.println("                                          Optionally a destination folder");
     pw.println("                                          for the tarball can be");
     pw.println("                                          specified.");
+    pw.println(" -express <arg>                           Works with -upgrade option to");
+    pw.println("                                          perform express upgrade.  It");
+    pw.println("                                          requires the upgraded");
+    pw.println("                                          application specification file.");
     pw.println(" -finalize                                Works with -upgrade option to");
     pw.println("                                          finalize the upgrade.");
     pw.println(" -flex <Application Name or ID>           Changes number of running");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e557c6bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
index 3fb4778..232666d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -288,4 +289,15 @@ public abstract class AppAdminClient extends CompositeService {
       List<String> components, String version, List<String> containerStates)
       throws IOException, YarnException;
 
+  /**
+   * Express upgrade a long running service.
+   *
+   * @param appName  the name of the application
+   * @param fileName specification of application upgrade to save.
+   * @return exit code
+   */
+  @Public
+  @Unstable
+  public abstract int actionUpgradeExpress(String appName, File fileName)
+      throws IOException, YarnException;
 }


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


[45/50] [abbrv] hadoop git commit: Merge commit 'b67ae6d9d741e79ccf2bd6f08a37fce070e6ad77' into HDFS-12943

Posted by sh...@apache.org.
Merge commit 'b67ae6d9d741e79ccf2bd6f08a37fce070e6ad77' into HDFS-12943

# Conflicts:
#	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
#	hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
#	hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
#	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java


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

Branch: refs/heads/HDFS-12943
Commit: 066afcf291d84d4215b7b447937d9e55f052239e
Parents: 90e61bf 3ac07b7
Author: Konstantin V Shvachko <sh...@apache.org>
Authored: Sat Aug 25 17:13:21 2018 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Sat Aug 25 17:13:21 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   19 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   34 +
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |   10 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   14 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   32 +
 .../ClientNamenodeProtocolTranslatorPB.java     |   13 +
 .../src/main/proto/ClientNamenodeProtocol.proto |   11 +-
 .../router/RouterNamenodeProtocol.java          |    7 +
 .../federation/router/RouterRpcServer.java      |   12 +
 .../hadoop-hdfs/src/main/bin/hdfs               |    7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   44 +
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    |    2 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |   20 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |    4 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |   18 +
 .../NamenodeProtocolTranslatorPB.java           |   15 +
 .../hdfs/server/balancer/NameNodeConnector.java |   36 +-
 .../blockmanagement/BlockInfoStriped.java       |   10 +
 .../server/blockmanagement/BlockManager.java    |   72 +
 .../blockmanagement/DatanodeDescriptor.java     |    1 -
 .../server/blockmanagement/DatanodeManager.java |   24 +-
 .../hdfs/server/common/HdfsServerConstants.java |    5 +
 .../hdfs/server/common/sps/BlockDispatcher.java |  186 ++
 .../sps/BlockMovementAttemptFinished.java       |  100 ++
 .../server/common/sps/BlockMovementStatus.java  |   53 +
 .../common/sps/BlockStorageMovementTracker.java |   91 +
 .../sps/BlocksMovementsStatusHandler.java       |   39 +
 .../hdfs/server/common/sps/package-info.java    |   27 +
 .../hdfs/server/datanode/BPServiceActor.java    |    2 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |    2 +-
 .../apache/hadoop/hdfs/server/mover/Mover.java  |    7 +-
 .../namenode/FSDirSatisfyStoragePolicyOp.java   |  157 ++
 .../hdfs/server/namenode/FSDirXAttrOp.java      |    7 +
 .../hdfs/server/namenode/FSDirectory.java       |   19 +
 .../hdfs/server/namenode/FSNamesystem.java      |   97 +-
 .../hdfs/server/namenode/FSTreeTraverser.java   |    4 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |   48 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   47 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   10 +
 .../server/namenode/ReencryptionHandler.java    |    3 +-
 .../namenode/sps/BlockMoveTaskHandler.java      |   43 +
 .../namenode/sps/BlockMovementListener.java     |   40 +
 .../sps/BlockStorageMovementAttemptedItems.java |  313 ++++
 .../sps/BlockStorageMovementNeeded.java         |  346 ++++
 .../hdfs/server/namenode/sps/Context.java       |  141 ++
 .../namenode/sps/DatanodeCacheManager.java      |  121 ++
 .../hdfs/server/namenode/sps/FileCollector.java |   43 +
 .../hdfs/server/namenode/sps/ItemInfo.java      |   82 +
 .../hdfs/server/namenode/sps/SPSService.java    |  127 ++
 .../namenode/sps/StoragePolicySatisfier.java    | 1226 +++++++++++++
 .../sps/StoragePolicySatisfyManager.java        |  293 ++++
 .../hdfs/server/namenode/sps/package-info.java  |   28 +
 .../protocol/BlockStorageMovementCommand.java   |  136 ++
 .../BlocksStorageMoveAttemptFinished.java       |   48 +
 .../hdfs/server/protocol/DatanodeProtocol.java  |    2 +
 .../server/protocol/DropSPSWorkCommand.java     |   36 +
 .../hdfs/server/protocol/NamenodeProtocol.java  |   10 +-
 .../sps/ExternalSPSBlockMoveTaskHandler.java    |  223 +++
 .../hdfs/server/sps/ExternalSPSContext.java     |  211 +++
 .../sps/ExternalSPSFilePathCollector.java       |  174 ++
 .../sps/ExternalStoragePolicySatisfier.java     |  117 ++
 .../hadoop/hdfs/server/sps/package-info.java    |   28 +
 .../hadoop/hdfs/tools/StoragePolicyAdmin.java   |   61 +-
 .../src/main/proto/DatanodeProtocol.proto       |    1 +
 .../src/main/proto/NamenodeProtocol.proto       |   21 +
 .../src/main/resources/hdfs-default.xml         |  125 ++
 .../src/site/markdown/ArchivalStorage.md        |   71 +-
 .../src/site/markdown/HDFSCommands.md           |    2 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  112 ++
 .../SimpleBlocksMovementsStatusHandler.java     |   88 +
 .../hadoop/hdfs/server/mover/TestMover.java     |   73 +-
 .../hdfs/server/mover/TestStorageMover.java     |    2 +
 .../hdfs/server/namenode/TestDeadDatanode.java  |    2 +-
 .../namenode/TestNameNodeReconfigure.java       |  130 ++
 .../TestPersistentStoragePolicySatisfier.java   |  531 ++++++
 .../TestStoragePolicySatisfierWithHA.java       |  112 ++
 .../TestBlockStorageMovementAttemptedItems.java |  220 +++
 ...stStoragePolicySatisfierWithStripedFile.java |  581 ++++++
 .../sps/TestExternalStoragePolicySatisfier.java | 1650 ++++++++++++++++++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |    2 +-
 .../hdfs/tools/TestStoragePolicyCommands.java   |   10 +-
 .../TestStoragePolicySatisfyAdminCommands.java  |  112 ++
 82 files changed, 8953 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 84a875a,e8c881b..9e456a5
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@@ -1789,13 -1744,16 +1789,27 @@@ public interface ClientProtocol 
        EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException;
  
    /**
 +   * Called by client to wait until the server has reached the state id of the
 +   * client. The client and server state id are given by client side and server
 +   * side alignment context respectively. This can be a blocking call.
 +   *
 +   * @throws IOException
 +   */
 +  @Idempotent
 +  @ReadOnly
 +  void msync() throws IOException;
++
++  /**
+    * Satisfy the storage policy for a file/directory.
+    * @param path Path of an existing file/directory.
+    * @throws AccessControlException If access is denied.
+    * @throws org.apache.hadoop.fs.UnresolvedLinkException if <code>src</code>
+    *           contains a symlink.
+    * @throws java.io.FileNotFoundException If file/dir <code>src</code> is not
+    *           found.
+    * @throws org.apache.hadoop.hdfs.server.namenode.SafeModeException append not
+    *           allowed in safemode.
+    */
+   @AtMostOnce
+   void satisfyStoragePolicy(String path) throws IOException;
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 088a9aa,e4bca51..8ca199f
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@@ -1946,12 -1948,13 +1949,22 @@@ public class ClientNamenodeProtocolTran
    }
  
    @Override
 +  public void msync() throws IOException {
 +    MsyncRequestProto.Builder req = MsyncRequestProto.newBuilder();
 +    try {
 +      rpcProxy.msync(null, req.build());
 +    } catch (ServiceException e) {
 +      throw ProtobufHelper.getRemoteException(e);
 +    }
 +  }
++
+   public void satisfyStoragePolicy(String src) throws IOException {
+     SatisfyStoragePolicyRequestProto req =
+         SatisfyStoragePolicyRequestProto.newBuilder().setSrc(src).build();
+     try {
+       rpcProxy.satisfyStoragePolicy(null, req);
+     } catch (ServiceException e) {
+       throw ProtobufHelper.getRemoteException(e);
+     }
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 247e54b,49ea3f3..55113a1
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@@ -831,12 -830,14 +830,20 @@@ message ListOpenFilesResponseProto 
    repeated OpenFilesTypeProto types = 3;
  }
  
 +message MsyncRequestProto {
 +}
 +
 +message MsyncResponseProto {
 +}
 +
+ message SatisfyStoragePolicyRequestProto {
+   required string src = 1;
+ }
+ 
+ message SatisfyStoragePolicyResponseProto {
+ 
+ }
+ 
  service ClientNamenodeProtocol {
    rpc getBlockLocations(GetBlockLocationsRequestProto)
        returns(GetBlockLocationsResponseProto);
@@@ -1023,6 -1024,6 +1030,8 @@@
        returns(GetQuotaUsageResponseProto);
    rpc listOpenFiles(ListOpenFilesRequestProto)
        returns(ListOpenFilesResponseProto);
 +  rpc msync(MsyncRequestProto)
 +      returns(MsyncResponseProto);
+   rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
+       returns(SatisfyStoragePolicyResponseProto);
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 81dc0fa,e51529e..8a4d4a1
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@@ -1890,13 -1894,14 +1896,25 @@@ public class ClientNamenodeProtocolServ
    }
  
    @Override
 +  public MsyncResponseProto msync(RpcController controller,
 +      MsyncRequestProto req) throws ServiceException {
 +    try {
 +      server.msync();
 +      return MsyncResponseProto.newBuilder().build();
 +    } catch (IOException e) {
 +      throw new ServiceException(e);
 +    }
 +  }
++
++  @Override
+   public SatisfyStoragePolicyResponseProto satisfyStoragePolicy(
+       RpcController controller,
+       SatisfyStoragePolicyRequestProto request) throws ServiceException {
+     try {
+       server.satisfyStoragePolicy(request.getSrc());
+     } catch (IOException e) {
+       throw new ServiceException(e);
+     }
+     return VOID_SATISFYSTORAGEPOLICY_RESPONSE;
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/066afcf2/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------


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


[43/50] [abbrv] hadoop git commit: HDDS-364. Update open container replica information in SCM during DN register. Contributed by Ajay Kumar.

Posted by sh...@apache.org.
HDDS-364. Update open container replica information in SCM during DN register. Contributed by Ajay Kumar.


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

Branch: refs/heads/HDFS-12943
Commit: a5eba25506a4ca7ac9efa9b60b204c8cf1aa4160
Parents: 8563fd6
Author: Márton Elek <el...@apache.org>
Authored: Fri Aug 24 22:27:43 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Fri Aug 24 22:42:51 2018 +0200

----------------------------------------------------------------------
 .../hdds/scm/container/ContainerMapping.java    | 25 ++++++++++++++------
 .../scm/container/ContainerReportHandler.java   |  3 ++-
 .../hadoop/hdds/scm/container/Mapping.java      |  2 +-
 .../scm/server/SCMDatanodeProtocolServer.java   |  2 +-
 .../scm/container/TestContainerMapping.java     | 19 ++++++++++++---
 .../container/closer/TestContainerCloser.java   |  2 +-
 .../commandhandler/TestBlockDeletion.java       |  3 ++-
 7 files changed, 41 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index ba95764..b000bfd 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -505,15 +505,26 @@ public class ContainerMapping implements Mapping {
    */
   @Override
   public void processContainerReports(DatanodeDetails datanodeDetails,
-                                      ContainerReportsProto reports)
+      ContainerReportsProto reports, boolean isRegisterCall)
       throws IOException {
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
         containerInfos = reports.getReportsList();
     PendingDeleteStatusList pendingDeleteStatusList =
         new PendingDeleteStatusList(datanodeDetails);
-    for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo contInfo :
         containerInfos) {
-      byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
+      // Update replica info during registration process.
+      if (isRegisterCall) {
+        try {
+          getStateManager().addContainerReplica(ContainerID.
+              valueof(contInfo.getContainerID()), datanodeDetails);
+        } catch (Exception ex) {
+          // Continue to next one after logging the error.
+          LOG.error("Error while adding replica for containerId {}.",
+              contInfo.getContainerID(), ex);
+        }
+      }
+      byte[] dbKey = Longs.toByteArray(contInfo.getContainerID());
       lock.lock();
       try {
         byte[] containerBytes = containerStore.get(dbKey);
@@ -522,12 +533,12 @@ public class ContainerMapping implements Mapping {
               HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
 
           HddsProtos.SCMContainerInfo newState =
-              reconcileState(datanodeState, knownState, datanodeDetails);
+              reconcileState(contInfo, knownState, datanodeDetails);
 
-          if (knownState.getDeleteTransactionId() > datanodeState
+          if (knownState.getDeleteTransactionId() > contInfo
               .getDeleteTransactionId()) {
             pendingDeleteStatusList
-                .addPendingDeleteStatus(datanodeState.getDeleteTransactionId(),
+                .addPendingDeleteStatus(contInfo.getDeleteTransactionId(),
                     knownState.getDeleteTransactionId(),
                     knownState.getContainerID());
           }
@@ -558,7 +569,7 @@ public class ContainerMapping implements Mapping {
           LOG.error("Error while processing container report from datanode :" +
                   " {}, for container: {}, reason: container doesn't exist in" +
                   "container database.", datanodeDetails,
-              datanodeState.getContainerID());
+              contInfo.getContainerID());
         }
       } finally {
         lock.unlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
index b26eed2..5a9e726 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
@@ -84,7 +84,8 @@ public class ContainerReportHandler implements
     try {
 
       //update state in container db and trigger close container events
-      containerMapping.processContainerReports(datanodeOrigin, containerReport);
+      containerMapping
+          .processContainerReports(datanodeOrigin, containerReport, false);
 
       Set<ContainerID> containerIds = containerReport.getReportsList().stream()
           .map(containerProto -> containerProto.getContainerID())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
index ac84be4..f4b5bb2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/Mapping.java
@@ -115,7 +115,7 @@ public interface Mapping extends Closeable {
    * @param reports Container report
    */
   void processContainerReports(DatanodeDetails datanodeDetails,
-                               ContainerReportsProto reports)
+      ContainerReportsProto reports, boolean isRegisterCall)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index 0d34787..9215803 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -196,7 +196,7 @@ public class SCMDatanodeProtocolServer implements
     if (registeredCommand.getError()
         == SCMRegisteredResponseProto.ErrorCode.success) {
       scm.getScmContainerManager().processContainerReports(datanodeDetails,
-          containerReportsProto);
+          containerReportsProto, true);
     }
     return getRegisteredResponse(registeredCommand);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
index 1e9c35b..b0b39f1 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
@@ -242,7 +242,7 @@ public class TestContainerMapping {
   }
 
   @Test
-  public void testFullContainerReport() throws IOException {
+  public void testFullContainerReport() throws Exception {
     ContainerInfo info = createContainer();
     DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
@@ -266,13 +266,26 @@ public class TestContainerMapping {
         .newBuilder();
     crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(datanodeDetails, crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build(), false);
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());
     Assert.assertEquals(100000000L,
         updatedContainer.getNumberOfKeys());
     Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
+
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
+      LambdaTestUtils.intercept(SCMException.class, "No entry "
+          + "exist for containerId:", () -> mapping.getStateManager()
+          .getContainerReplicas(ContainerID.valueof(c.getContainerID())));
+    }
+
+    mapping.processContainerReports(TestUtils.randomDatanodeDetails(),
+        crBuilder.build(), true);
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
+      Assert.assertTrue(mapping.getStateManager().getContainerReplicas(
+          ContainerID.valueof(c.getContainerID())).size() > 0);
+    }
   }
 
   @Test
@@ -301,7 +314,7 @@ public class TestContainerMapping {
         ContainerReportsProto.newBuilder();
     crBuilder.addAllReports(reports);
 
-    mapping.processContainerReports(datanodeDetails, crBuilder.build());
+    mapping.processContainerReports(datanodeDetails, crBuilder.build(), false);
 
     ContainerInfo updatedContainer =
         mapping.getContainer(info.getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
index 608bb92..210df08 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/closer/TestContainerCloser.java
@@ -223,6 +223,6 @@ public class TestContainerCloser {
         .setDeleteTransactionId(0);
     reports.addReports(ciBuilder);
     mapping.processContainerReports(TestUtils.randomDatanodeDetails(),
-        reports.build());
+        reports.build(), false);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5eba255/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 94cdf61..5794151 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -187,7 +187,8 @@ public class TestBlockDeletion {
 
     logCapturer.clearOutput();
     scm.getScmContainerManager().processContainerReports(
-        cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport);
+        cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport,
+        false);
     // wait for event to be handled by event handler
     Thread.sleep(1000);
     String output = logCapturer.getOutput();


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


[15/50] [abbrv] hadoop git commit: HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun.

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/6425ed27/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 29f32a6..fe54993 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -33,16 +33,12 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -54,7 +50,6 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -64,7 +59,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
@@ -93,7 +87,6 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
-import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -101,8 +94,8 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
@@ -167,11 +160,6 @@ public class RouterRpcServer extends AbstractService
   /** Configuration for the RPC server. */
   private Configuration conf;
 
-  /** Identifier for the super user. */
-  private final String superUser;
-  /** Identifier for the super group. */
-  private final String superGroup;
-
   /** Router using this RPC server. */
   private final Router router;
 
@@ -199,11 +187,10 @@ public class RouterRpcServer extends AbstractService
   // Modules implementing groups of RPC calls
   /** Router Quota calls. */
   private final Quota quotaCall;
-  /** Erasure coding calls. */
-  private final ErasureCoding erasureCoding;
   /** NamenodeProtocol calls. */
   private final RouterNamenodeProtocol nnProto;
-
+  /** ClientProtocol calls. */
+  private final RouterClientProtocol clientProto;
 
   /**
    * Construct a router RPC server.
@@ -223,12 +210,6 @@ public class RouterRpcServer extends AbstractService
     this.namenodeResolver = nnResolver;
     this.subclusterResolver = fileResolver;
 
-    // User and group for reporting
-    this.superUser = System.getProperty("user.name");
-    this.superGroup = this.conf.get(
-        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
-        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
-
     // RPC server settings
     int handlerCount = this.conf.getInt(DFS_ROUTER_HANDLER_COUNT_KEY,
         DFS_ROUTER_HANDLER_COUNT_DEFAULT);
@@ -315,8 +296,8 @@ public class RouterRpcServer extends AbstractService
 
     // Initialize modules
     this.quotaCall = new Quota(this.router, this);
-    this.erasureCoding = new ErasureCoding(this);
     this.nnProto = new RouterNamenodeProtocol(this);
+    this.clientProto = new RouterClientProtocol(conf, this);
   }
 
   @Override
@@ -371,6 +352,13 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
+   * Get the active namenode resolver
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return namenodeResolver;
+  }
+
+  /**
    * Get the RPC monitor and metrics.
    *
    * @return RPC monitor and metrics.
@@ -411,7 +399,7 @@ public class RouterRpcServer extends AbstractService
    *                           client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  protected void checkOperation(OperationCategory op, boolean supported)
+  void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -433,7 +421,7 @@ public class RouterRpcServer extends AbstractService
    * @throws SafeModeException If the Router is in safe mode and cannot serve
    *                           client requests.
    */
-  protected void checkOperation(OperationCategory op)
+  void checkOperation(OperationCategory op)
       throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
@@ -464,58 +452,44 @@ public class RouterRpcServer extends AbstractService
     }
   }
 
+  /**
+   * Get the name of the method that is calling this function.
+   *
+   * @return Name of the method calling this function.
+   */
+  static String getMethodName() {
+    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
+    String methodName = stack[3].getMethodName();
+    return methodName;
+  }
+
   @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
-  }
-
-  /**
-   * The the delegation token from each name service.
-   * @param renewer
-   * @return Name service -> Token.
-   * @throws IOException
-   */
-  public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
-      getDelegationTokens(Text renewer) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
+    return clientProto.getDelegationToken(renewer);
   }
 
   @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return 0;
+    return clientProto.renewDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.cancelDelegationToken(token);
   }
 
   @Override // ClientProtocol
   public LocatedBlocks getBlockLocations(String src, final long offset,
       final long length) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
-        new Class<?>[] {String.class, long.class, long.class},
-        new RemoteParam(), offset, length);
-    return (LocatedBlocks) rpcClient.invokeSequential(locations, remoteMethod,
-        LocatedBlocks.class, null);
+    return clientProto.getBlockLocations(src, offset, length);
   }
 
   @Override // ClientProtocol
   public FsServerDefaults getServerDefaults() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("getServerDefaults");
-    String ns = subclusterResolver.getDefaultNamespace();
-    return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
+    return clientProto.getServerDefaults();
   }
 
   @Override // ClientProtocol
@@ -524,44 +498,8 @@ public class RouterRpcServer extends AbstractService
       boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    if (createParent && isPathAll(src)) {
-      int index = src.lastIndexOf(Path.SEPARATOR);
-      String parent = src.substring(0, index);
-      LOG.debug("Creating {} requires creating parent {}", src, parent);
-      FsPermission parentPermissions = getParentPermission(masked);
-      boolean success = mkdirs(parent, parentPermissions, createParent);
-      if (!success) {
-        // This shouldn't happen as mkdirs returns true or exception
-        LOG.error("Couldn't create parents for {}", src);
-      }
-    }
-
-    RemoteLocation createLocation = getCreateLocation(src);
-    RemoteMethod method = new RemoteMethod("create",
-        new Class<?>[] {String.class, FsPermission.class, String.class,
-                        EnumSetWritable.class, boolean.class, short.class,
-                        long.class, CryptoProtocolVersion[].class,
-                        String.class},
-        createLocation.getDest(), masked, clientName, flag, createParent,
+    return clientProto.create(src, masked, clientName, flag, createParent,
         replication, blockSize, supportedVersions, ecPolicyName);
-    return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
-  }
-
-  /**
-   * Get the permissions for the parent of a child with given permissions.
-   * Add implicit u+wx permission for parent. This is based on
-   * @{FSDirMkdirOp#addImplicitUwx}.
-   * @param mask The permission mask of the child.
-   * @return The permission mask of the parent.
-   */
-  private static FsPermission getParentPermission(final FsPermission mask) {
-    FsPermission ret = new FsPermission(
-        mask.getUserAction().or(FsAction.WRITE_EXECUTE),
-        mask.getGroupAction(),
-        mask.getOtherAction());
-    return ret;
   }
 
   /**
@@ -572,7 +510,7 @@ public class RouterRpcServer extends AbstractService
    * @return The remote location for this file.
    * @throws IOException If the file has no creation location.
    */
-  protected RemoteLocation getCreateLocation(final String src)
+  RemoteLocation getCreateLocation(final String src)
       throws IOException {
 
     final List<RemoteLocation> locations = getLocationsForPath(src, true);
@@ -613,100 +551,45 @@ public class RouterRpcServer extends AbstractService
     return createLocation;
   }
 
-  // Medium
   @Override // ClientProtocol
   public LastBlockWithStatus append(String src, final String clientName,
       final EnumSetWritable<CreateFlag> flag) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("append",
-        new Class<?>[] {String.class, String.class, EnumSetWritable.class},
-        new RemoteParam(), clientName, flag);
-    return rpcClient.invokeSequential(
-        locations, method, LastBlockWithStatus.class, null);
+    return clientProto.append(src, clientName, flag);
   }
 
-  // Low
   @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("recoverLease",
-        new Class<?>[] {String.class, String.class}, new RemoteParam(),
-        clientName);
-    Object result = rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE);
-    return (boolean) result;
+    return clientProto.recoverLease(src, clientName);
   }
 
   @Override // ClientProtocol
   public boolean setReplication(String src, short replication)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setReplication",
-        new Class<?>[] {String.class, short.class}, new RemoteParam(),
-        replication);
-    Object result = rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE);
-    return (boolean) result;
+    return clientProto.setReplication(src, replication);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setStoragePolicy",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), policyName);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.setStoragePolicy(src, policyName);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("getStoragePolicies");
-    String ns = subclusterResolver.getDefaultNamespace();
-    return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
+    return clientProto.getStoragePolicies();
   }
 
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setPermission",
-        new Class<?>[] {String.class, FsPermission.class},
-        new RemoteParam(), permissions);
-    if (isPathAll(src)) {
-      rpcClient.invokeConcurrent(locations, method);
-    } else {
-      rpcClient.invokeSequential(locations, method);
-    }
+    clientProto.setPermission(src, permissions);
   }
 
   @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setOwner",
-        new Class<?>[] {String.class, String.class, String.class},
-        new RemoteParam(), username, groupname);
-    if (isPathAll(src)) {
-      rpcClient.invokeConcurrent(locations, method);
-    } else {
-      rpcClient.invokeSequential(locations, method);
-    }
+    clientProto.setOwner(src, username, groupname);
   }
 
   /**
@@ -718,18 +601,8 @@ public class RouterRpcServer extends AbstractService
       ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
       String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("addBlock",
-        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
-                        DatanodeInfo[].class, long.class, String[].class,
-                        EnumSet.class},
-        new RemoteParam(), clientName, previous, excludedNodes, fileId,
-        favoredNodes, addBlockFlags);
-    // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
-    return (LocatedBlock) rpcClient.invokeSequential(
-        locations, method, LocatedBlock.class, null);
+    return clientProto.addBlock(src, clientName, previous, excludedNodes,
+        fileId, favoredNodes, addBlockFlags);
   }
 
   /**
@@ -742,55 +615,26 @@ public class RouterRpcServer extends AbstractService
       final String[] existingStorageIDs, final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName)
           throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
-        new Class<?>[] {String.class, long.class, ExtendedBlock.class,
-                        DatanodeInfo[].class, String[].class,
-                        DatanodeInfo[].class, int.class, String.class},
-        new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
-        numAdditionalNodes, clientName);
-    return (LocatedBlock) rpcClient.invokeSequential(
-        locations, method, LocatedBlock.class, null);
+    return clientProto.getAdditionalDatanode(src, fileId, blk, existings,
+        existingStorageIDs, excludes, numAdditionalNodes, clientName);
   }
 
   @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
       String holder) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("abandonBlock",
-        new Class<?>[] {ExtendedBlock.class, long.class, String.class,
-                        String.class},
-        b, fileId, new RemoteParam(), holder);
-    rpcClient.invokeSingle(b, method);
+    clientProto.abandonBlock(b, fileId, src, holder);
   }
 
   @Override // ClientProtocol
   public boolean complete(String src, String clientName, ExtendedBlock last,
       long fileId) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("complete",
-        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
-                        long.class},
-        new RemoteParam(), clientName, last, fileId);
-    // Complete can return true/false, so don't expect a result
-    return ((Boolean) rpcClient.invokeSequential(
-        locations, method, Boolean.class, null)).booleanValue();
+    return clientProto.complete(src, clientName, last, fileId);
   }
 
   @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(
       ExtendedBlock block, String clientName) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
-        new Class<?>[] {ExtendedBlock.class, String.class},
-        block, clientName);
-    return (LocatedBlock) rpcClient.invokeSingle(block, method);
+    return clientProto.updateBlockForPipeline(block, clientName);
   }
 
   /**
@@ -801,462 +645,91 @@ public class RouterRpcServer extends AbstractService
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
           throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("updatePipeline",
-        new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
-                        DatanodeID[].class, String[].class},
-        clientName, oldBlock, newBlock, newNodes, newStorageIDs);
-    rpcClient.invokeSingle(oldBlock, method);
+    clientProto.updatePipeline(clientName, oldBlock, newBlock, newNodes,
+        newStorageIDs);
   }
 
   @Override // ClientProtocol
   public long getPreferredBlockSize(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return ((Long) rpcClient.invokeSequential(
-        locations, method, Long.class, null)).longValue();
-  }
-
-  /**
-   * Determines combinations of eligible src/dst locations for a rename. A
-   * rename cannot change the namespace. Renames are only allowed if there is an
-   * eligible dst location in the same namespace as the source.
-   *
-   * @param srcLocations List of all potential source destinations where the
-   *          path may be located. On return this list is trimmed to include
-   *          only the paths that have corresponding destinations in the same
-   *          namespace.
-   * @param dst The destination path
-   * @return A map of all eligible source namespaces and their corresponding
-   *         replacement value.
-   * @throws IOException If the dst paths could not be determined.
-   */
-  private RemoteParam getRenameDestinations(
-      final List<RemoteLocation> srcLocations, final String dst)
-          throws IOException {
-
-    final List<RemoteLocation> dstLocations = getLocationsForPath(dst, true);
-    final Map<RemoteLocation, String> dstMap = new HashMap<>();
-
-    Iterator<RemoteLocation> iterator = srcLocations.iterator();
-    while (iterator.hasNext()) {
-      RemoteLocation srcLocation = iterator.next();
-      RemoteLocation eligibleDst =
-          getFirstMatchingLocation(srcLocation, dstLocations);
-      if (eligibleDst != null) {
-        // Use this dst for this source location
-        dstMap.put(srcLocation, eligibleDst.getDest());
-      } else {
-        // This src destination is not valid, remove from the source list
-        iterator.remove();
-      }
-    }
-    return new RemoteParam(dstMap);
-  }
-
-  /**
-   * Get first matching location.
-   *
-   * @param location Location we are looking for.
-   * @param locations List of locations.
-   * @return The first matchin location in the list.
-   */
-  private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
-      List<RemoteLocation> locations) {
-    for (RemoteLocation loc : locations) {
-      if (loc.getNameserviceId().equals(location.getNameserviceId())) {
-        // Return first matching location
-        return loc;
-      }
-    }
-    return null;
+    return clientProto.getPreferredBlockSize(src);
   }
 
   @Deprecated
   @Override // ClientProtocol
   public boolean rename(final String src, final String dst)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> srcLocations =
-        getLocationsForPath(src, true, false);
-    // srcLocations may be trimmed by getRenameDestinations()
-    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
-    RemoteParam dstParam = getRenameDestinations(locs, dst);
-    if (locs.isEmpty()) {
-      throw new IOException(
-          "Rename of " + src + " to " + dst + " is not allowed," +
-          " no eligible destination in the same namespace was found.");
-    }
-    RemoteMethod method = new RemoteMethod("rename",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), dstParam);
-    return ((Boolean) rpcClient.invokeSequential(
-        locs, method, Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.rename(src, dst);
   }
 
   @Override // ClientProtocol
   public void rename2(final String src, final String dst,
       final Options.Rename... options) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> srcLocations =
-        getLocationsForPath(src, true, false);
-    // srcLocations may be trimmed by getRenameDestinations()
-    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
-    RemoteParam dstParam = getRenameDestinations(locs, dst);
-    if (locs.isEmpty()) {
-      throw new IOException(
-          "Rename of " + src + " to " + dst + " is not allowed," +
-          " no eligible destination in the same namespace was found.");
-    }
-    RemoteMethod method = new RemoteMethod("rename2",
-        new Class<?>[] {String.class, String.class, options.getClass()},
-        new RemoteParam(), dstParam, options);
-    rpcClient.invokeSequential(locs, method, null, null);
+    clientProto.rename2(src, dst, options);
   }
 
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // See if the src and target files are all in the same namespace
-    LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
-    if (targetBlocks == null) {
-      throw new IOException("Cannot locate blocks for target file - " + trg);
-    }
-    LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
-    String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
-    for (String source : src) {
-      LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
-      if (sourceBlocks == null) {
-        throw new IOException(
-            "Cannot located blocks for source file " + source);
-      }
-      String sourceBlockPoolId =
-          sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
-      if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
-        throw new IOException("Cannot concatenate source file " + source
-            + " because it is located in a different namespace"
-            + " with block pool id " + sourceBlockPoolId
-            + " from the target file with block pool id "
-            + targetBlockPoolId);
-      }
-    }
-
-    // Find locations in the matching namespace.
-    final RemoteLocation targetDestination =
-        getLocationForPath(trg, true, targetBlockPoolId);
-    String[] sourceDestinations = new String[src.length];
-    for (int i = 0; i < src.length; i++) {
-      String sourceFile = src[i];
-      RemoteLocation location =
-          getLocationForPath(sourceFile, true, targetBlockPoolId);
-      sourceDestinations[i] = location.getDest();
-    }
-    // Invoke
-    RemoteMethod method = new RemoteMethod("concat",
-        new Class<?>[] {String.class, String[].class},
-        targetDestination.getDest(), sourceDestinations);
-    rpcClient.invokeSingle(targetDestination, method);
+    clientProto.concat(trg, src);
   }
 
   @Override // ClientProtocol
   public boolean truncate(String src, long newLength, String clientName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("truncate",
-        new Class<?>[] {String.class, long.class, String.class},
-        new RemoteParam(), newLength, clientName);
-    return ((Boolean) rpcClient.invokeSequential(locations, method,
-        Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.truncate(src, newLength, clientName);
   }
 
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations =
-        getLocationsForPath(src, true, false);
-    RemoteMethod method = new RemoteMethod("delete",
-        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
-        recursive);
-    if (isPathAll(src)) {
-      return rpcClient.invokeAll(locations, method);
-    } else {
-      return rpcClient.invokeSequential(locations, method,
-          Boolean.class, Boolean.TRUE).booleanValue();
-    }
+    return clientProto.delete(src, recursive);
   }
 
   @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("mkdirs",
-        new Class<?>[] {String.class, FsPermission.class, boolean.class},
-        new RemoteParam(), masked, createParent);
-
-    // Create in all locations
-    if (isPathAll(src)) {
-      return rpcClient.invokeAll(locations, method);
-    }
-
-    if (locations.size() > 1) {
-      // Check if this directory already exists
-      try {
-        HdfsFileStatus fileStatus = getFileInfo(src);
-        if (fileStatus != null) {
-          // When existing, the NN doesn't return an exception; return true
-          return true;
-        }
-      } catch (IOException ioe) {
-        // Can't query if this file exists or not.
-        LOG.error("Error requesting file info for path {} while proxing mkdirs",
-            src, ioe);
-      }
-    }
-
-    RemoteLocation firstLocation = locations.get(0);
-    return ((Boolean) rpcClient.invokeSingle(firstLocation, method))
-        .booleanValue();
+    return clientProto.mkdirs(src, masked, createParent);
   }
 
   @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("renewLease",
-        new Class<?>[] {String.class}, clientName);
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, false, false);
+    clientProto.renewLease(clientName);
   }
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // Locate the dir and fetch the listing
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("getListing",
-        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
-        new RemoteParam(), startAfter, needLocation);
-    Map<RemoteLocation, DirectoryListing> listings =
-        rpcClient.invokeConcurrent(
-            locations, method, false, false, DirectoryListing.class);
-
-    Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
-    int totalRemainingEntries = 0;
-    int remainingEntries = 0;
-    boolean namenodeListingExists = false;
-    if (listings != null) {
-      // Check the subcluster listing with the smallest name
-      String lastName = null;
-      for (Entry<RemoteLocation, DirectoryListing> entry :
-          listings.entrySet()) {
-        RemoteLocation location = entry.getKey();
-        DirectoryListing listing = entry.getValue();
-        if (listing == null) {
-          LOG.debug("Cannot get listing from {}", location);
-        } else {
-          totalRemainingEntries += listing.getRemainingEntries();
-          HdfsFileStatus[] partialListing = listing.getPartialListing();
-          int length = partialListing.length;
-          if (length > 0) {
-            HdfsFileStatus lastLocalEntry = partialListing[length-1];
-            String lastLocalName = lastLocalEntry.getLocalName();
-            if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
-              lastName = lastLocalName;
-            }
-          }
-        }
-      }
-
-      // Add existing entries
-      for (Object value : listings.values()) {
-        DirectoryListing listing = (DirectoryListing) value;
-        if (listing != null) {
-          namenodeListingExists = true;
-          for (HdfsFileStatus file : listing.getPartialListing()) {
-            String filename = file.getLocalName();
-            if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
-              // Discarding entries further than the lastName
-              remainingEntries++;
-            } else {
-              nnListing.put(filename, file);
-            }
-          }
-          remainingEntries += listing.getRemainingEntries();
-        }
-      }
-    }
-
-    // Add mount points at this level in the tree
-    final List<String> children = subclusterResolver.getMountPoints(src);
-    if (children != null) {
-      // Get the dates for each mount point
-      Map<String, Long> dates = getMountPointDates(src);
-
-      // Create virtual folder with the mount name
-      for (String child : children) {
-        long date = 0;
-        if (dates != null && dates.containsKey(child)) {
-          date = dates.get(child);
-        }
-        // TODO add number of children
-        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
-
-        // This may overwrite existing listing entries with the mount point
-        // TODO don't add if already there?
-        nnListing.put(child, dirStatus);
-      }
-    }
-
-    if (!namenodeListingExists && nnListing.size() == 0) {
-      // NN returns a null object if the directory cannot be found and has no
-      // listing. If we didn't retrieve any NN listing data, and there are no
-      // mount points here, return null.
-      return null;
-    }
-
-    // Generate combined listing
-    HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
-    combinedData = nnListing.values().toArray(combinedData);
-    return new DirectoryListing(combinedData, remainingEntries);
+    return clientProto.getListing(src, startAfter, needLocation);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getFileInfo",
-        new Class<?>[] {String.class}, new RemoteParam());
-
-    HdfsFileStatus ret = null;
-    // If it's a directory, we check in all locations
-    if (isPathAll(src)) {
-      ret = getFileInfoAll(locations, method);
-    } else {
-      // Check for file information sequentially
-      ret = (HdfsFileStatus) rpcClient.invokeSequential(
-          locations, method, HdfsFileStatus.class, null);
-    }
-
-    // If there is no real path, check mount points
-    if (ret == null) {
-      List<String> children = subclusterResolver.getMountPoints(src);
-      if (children != null && !children.isEmpty()) {
-        Map<String, Long> dates = getMountPointDates(src);
-        long date = 0;
-        if (dates != null && dates.containsKey(src)) {
-          date = dates.get(src);
-        }
-        ret = getMountPointStatus(src, children.size(), date);
-      }
-    }
-
-    return ret;
-  }
-
-  /**
-   * Get the file info from all the locations.
-   *
-   * @param locations Locations to check.
-   * @param method The file information method to run.
-   * @return The first file info if it's a file, the directory if it's
-   *         everywhere.
-   * @throws IOException If all the locations throw an exception.
-   */
-  private HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
-      final RemoteMethod method) throws IOException {
-
-    // Get the file info from everybody
-    Map<RemoteLocation, HdfsFileStatus> results =
-        rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
-
-    // We return the first file
-    HdfsFileStatus dirStatus = null;
-    for (RemoteLocation loc : locations) {
-      HdfsFileStatus fileStatus = results.get(loc);
-      if (fileStatus != null) {
-        if (!fileStatus.isDirectory()) {
-          return fileStatus;
-        } else if (dirStatus == null) {
-          dirStatus = fileStatus;
-        }
-      }
-    }
-    return dirStatus;
+    return clientProto.getFileInfo(src);
   }
 
   @Override // ClientProtocol
   public boolean isFileClosed(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("isFileClosed",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return ((Boolean) rpcClient.invokeSequential(
-        locations, method, Boolean.class, Boolean.TRUE)).booleanValue();
+    return clientProto.isFileClosed(src);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getFileLinkInfo",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (HdfsFileStatus) rpcClient.invokeSequential(
-        locations, method, HdfsFileStatus.class, null);
+    return clientProto.getFileLinkInfo(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public HdfsLocatedFileStatus getLocatedFileInfo(String src,
       boolean needBlockToken) throws IOException {
-    checkOperation(OperationCategory.READ);
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
-        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
-        Boolean.valueOf(needBlockToken));
-    return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
-        locations, method, HdfsFileStatus.class, null);
+    return clientProto.getLocatedFileInfo(src, needBlockToken);
   }
 
   @Override // ClientProtocol
   public long[] getStats() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("getStats");
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, long[]> results =
-        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
-    long[] combinedData = new long[STATS_ARRAY_LENGTH];
-    for (long[] data : results.values()) {
-      for (int i = 0; i < combinedData.length && i < data.length; i++) {
-        if (data[i] >= 0) {
-          combinedData[i] += data[i];
-        }
-      }
-    }
-    return combinedData;
+    return clientProto.getStats();
   }
 
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-    return getDatanodeReport(type, true, 0);
+    return clientProto.getDatanodeReport(type);
   }
 
   /**
@@ -1305,29 +778,7 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public DatanodeStorageReport[] getDatanodeStorageReport(
       DatanodeReportType type) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    Map<String, DatanodeStorageReport[]> dnSubcluster =
-        getDatanodeStorageReportMap(type);
-
-    // Avoid repeating machines in multiple subclusters
-    Map<String, DatanodeStorageReport> datanodesMap = new LinkedHashMap<>();
-    for (DatanodeStorageReport[] dns : dnSubcluster.values()) {
-      for (DatanodeStorageReport dn : dns) {
-        DatanodeInfo dnInfo = dn.getDatanodeInfo();
-        String nodeId = dnInfo.getXferAddr();
-        if (!datanodesMap.containsKey(nodeId)) {
-          datanodesMap.put(nodeId, dn);
-        }
-        // TODO merge somehow, right now it just takes the first one
-      }
-    }
-
-    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
-    DatanodeStorageReport[] combinedData =
-        new DatanodeStorageReport[datanodes.size()];
-    combinedData = datanodes.toArray(combinedData);
-    return combinedData;
+    return clientProto.getDatanodeStorageReport(type);
   }
 
   /**
@@ -1360,740 +811,388 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action, boolean isChecked)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // Set safe mode in all the name spaces
-    RemoteMethod method = new RemoteMethod("setSafeMode",
-        new Class<?>[] {SafeModeAction.class, boolean.class},
-        action, isChecked);
-    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> results =
-        rpcClient.invokeConcurrent(
-            nss, method, true, !isChecked, Boolean.class);
-
-    // We only report true if all the name space are in safe mode
-    int numSafemode = 0;
-    for (boolean safemode : results.values()) {
-      if (safemode) {
-        numSafemode++;
-      }
-    }
-    return numSafemode == results.size();
+    return clientProto.setSafeMode(action, isChecked);
   }
 
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("restoreFailedStorage",
-        new Class<?>[] {String.class}, arg);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class);
-
-    boolean success = true;
-    for (boolean s : ret.values()) {
-      if (!s) {
-        success = false;
-        break;
-      }
-    }
-    return success;
+    return clientProto.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("saveNamespace",
-        new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Boolean> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
-
-    boolean success = true;
-    for (boolean s : ret.values()) {
-      if (!s) {
-        success = false;
-        break;
-      }
-    }
-    return success;
+    return clientProto.saveNamespace(timeWindow, txGap);
   }
 
   @Override // ClientProtocol
   public long rollEdits() throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Long> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
-
-    // Return the maximum txid
-    long txid = 0;
-    for (long t : ret.values()) {
-      if (t > txid) {
-        txid = t;
-      }
-    }
-    return txid;
+    return clientProto.rollEdits();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, true);
+    clientProto.refreshNodes();
   }
 
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("finalizeUpgrade",
-        new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public boolean upgradeStatus() throws IOException {
-    String methodName = getMethodName();
-    throw new UnsupportedOperationException(
-        "Operation \"" + methodName + "\" is not supported");
+    return clientProto.upgradeStatus();
   }
 
   @Override // ClientProtocol
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod("rollingUpgrade",
-        new Class<?>[] {RollingUpgradeAction.class}, action);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
-        rpcClient.invokeConcurrent(
-            nss, method, true, false, RollingUpgradeInfo.class);
-
-    // Return the first rolling upgrade info
-    RollingUpgradeInfo info = null;
-    for (RollingUpgradeInfo infoNs : ret.values()) {
-      if (info == null && infoNs != null) {
-        info = infoNs;
-      }
-    }
-    return info;
+    return clientProto.rollingUpgrade(action);
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("metaSave",
-        new Class<?>[] {String.class}, filename);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.metaSave(filename);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(path, false);
-    RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), cookie);
-    return (CorruptFileBlocks) rpcClient.invokeSequential(
-        locations, method, CorruptFileBlocks.class, null);
+    return clientProto.listCorruptFileBlocks(path, cookie);
   }
 
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    checkOperation(OperationCategory.UNCHECKED);
-
-    RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
-        new Class<?>[] {Long.class}, bandwidth);
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    rpcClient.invokeConcurrent(nss, method, true, false);
+    clientProto.setBalancerBandwidth(bandwidth);
   }
 
   @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // Get the summaries from regular files
-    Collection<ContentSummary> summaries = new LinkedList<>();
-    FileNotFoundException notFoundException = null;
-    try {
-      final List<RemoteLocation> locations = getLocationsForPath(path, false);
-      RemoteMethod method = new RemoteMethod("getContentSummary",
-          new Class<?>[] {String.class}, new RemoteParam());
-      Map<RemoteLocation, ContentSummary> results =
-          rpcClient.invokeConcurrent(
-              locations, method, false, false, ContentSummary.class);
-      summaries.addAll(results.values());
-    } catch (FileNotFoundException e) {
-      notFoundException = e;
-    }
-
-    // Add mount points at this level in the tree
-    final List<String> children = subclusterResolver.getMountPoints(path);
-    if (children != null) {
-      for (String child : children) {
-        Path childPath = new Path(path, child);
-        try {
-          ContentSummary mountSummary = getContentSummary(childPath.toString());
-          if (mountSummary != null) {
-            summaries.add(mountSummary);
-          }
-        } catch (Exception e) {
-          LOG.error("Cannot get content summary for mount {}: {}",
-              childPath, e.getMessage());
-        }
-      }
-    }
-
-    // Throw original exception if no original nor mount points
-    if (summaries.isEmpty() && notFoundException != null) {
-      throw notFoundException;
-    }
-
-    return aggregateContentSummary(summaries);
-  }
-
-  /**
-   * Aggregate content summaries for each subcluster.
-   *
-   * @param summaries Collection of individual summaries.
-   * @return Aggregated content summary.
-   */
-  private ContentSummary aggregateContentSummary(
-      Collection<ContentSummary> summaries) {
-    if (summaries.size() == 1) {
-      return summaries.iterator().next();
-    }
-
-    long length = 0;
-    long fileCount = 0;
-    long directoryCount = 0;
-    long quota = 0;
-    long spaceConsumed = 0;
-    long spaceQuota = 0;
-
-    for (ContentSummary summary : summaries) {
-      length += summary.getLength();
-      fileCount += summary.getFileCount();
-      directoryCount += summary.getDirectoryCount();
-      quota += summary.getQuota();
-      spaceConsumed += summary.getSpaceConsumed();
-      spaceQuota += summary.getSpaceQuota();
-    }
-
-    ContentSummary ret = new ContentSummary.Builder()
-        .length(length)
-        .fileCount(fileCount)
-        .directoryCount(directoryCount)
-        .quota(quota)
-        .spaceConsumed(spaceConsumed)
-        .spaceQuota(spaceQuota)
-        .build();
-    return ret;
+    return clientProto.getContentSummary(path);
   }
 
   @Override // ClientProtocol
   public void fsync(String src, long fileId, String clientName,
       long lastBlockLength) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("fsync",
-        new Class<?>[] {String.class, long.class, String.class, long.class },
-        new RemoteParam(), fileId, clientName, lastBlockLength);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.fsync(src, fileId, clientName, lastBlockLength);
   }
 
   @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setTimes",
-        new Class<?>[] {String.class, long.class, long.class},
-        new RemoteParam(), mtime, atime);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setTimes(src, mtime, atime);
   }
 
   @Override // ClientProtocol
   public void createSymlink(String target, String link, FsPermission dirPerms,
       boolean createParent) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO Verify that the link location is in the same NS as the targets
-    final List<RemoteLocation> targetLocations =
-        getLocationsForPath(target, true);
-    final List<RemoteLocation> linkLocations =
-        getLocationsForPath(link, true);
-    RemoteLocation linkLocation = linkLocations.get(0);
-    RemoteMethod method = new RemoteMethod("createSymlink",
-        new Class<?>[] {String.class, String.class, FsPermission.class,
-                        boolean.class},
-        new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
-    rpcClient.invokeSequential(targetLocations, method);
+    clientProto.createSymlink(target, link, dirPerms, createParent);
   }
 
   @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    final List<RemoteLocation> locations = getLocationsForPath(path, true);
-    RemoteMethod method = new RemoteMethod("getLinkTarget",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (String) rpcClient.invokeSequential(
-        locations, method, String.class, null);
+    return clientProto.getLinkTarget(path);
   }
 
   @Override // Client Protocol
   public void allowSnapshot(String snapshotRoot) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.allowSnapshot(snapshotRoot);
   }
 
   @Override // Client Protocol
   public void disallowSnapshot(String snapshot) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.disallowSnapshot(snapshot);
   }
 
   @Override // ClientProtocol
   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.renameSnapshot(snapshotRoot, snapshotOldName, snapshotNewName);
   }
 
   @Override // Client Protocol
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshottableDirListing();
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshotDiffReport(
+        snapshotRoot, earlierSnapshotName, laterSnapshotName);
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReportListing getSnapshotDiffReportListing(
       String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
       byte[] startPath, int index) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getSnapshotDiffReportListing(snapshotRoot,
+        earlierSnapshotName, laterSnapshotName, startPath, index);
   }
 
   @Override // ClientProtocol
   public long addCacheDirective(CacheDirectiveInfo path,
       EnumSet<CacheFlag> flags) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return 0;
+    return clientProto.addCacheDirective(path, flags);
   }
 
   @Override // ClientProtocol
   public void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.modifyCacheDirective(directive, flags);
   }
 
   @Override // ClientProtocol
   public void removeCacheDirective(long id) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.removeCacheDirective(id);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
       long prevId, CacheDirectiveInfo filter) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listCacheDirectives(prevId, filter);
   }
 
   @Override // ClientProtocol
   public void addCachePool(CachePoolInfo info) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.addCachePool(info);
   }
 
   @Override // ClientProtocol
   public void modifyCachePool(CachePoolInfo info) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.modifyCachePool(info);
   }
 
   @Override // ClientProtocol
   public void removeCachePool(String cachePoolName) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.removeCachePool(cachePoolName);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listCachePools(prevKey);
   }
 
   @Override // ClientProtocol
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("modifyAclEntries",
-        new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.modifyAclEntries(src, aclSpec);
   }
 
   @Override // ClienProtocol
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeAclEntries",
-        new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method, null, null);
+    clientProto.removeAclEntries(src, aclSpec);
   }
 
   @Override // ClientProtocol
   public void removeDefaultAcl(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeDefaultAcl",
-        new Class<?>[] {String.class}, new RemoteParam());
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeDefaultAcl(src);
   }
 
   @Override // ClientProtocol
   public void removeAcl(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeAcl",
-        new Class<?>[] {String.class}, new RemoteParam());
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeAcl(src);
   }
 
   @Override // ClientProtocol
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod(
-        "setAcl", new Class<?>[] {String.class, List.class},
-        new RemoteParam(), aclSpec);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setAcl(src, aclSpec);
   }
 
   @Override // ClientProtocol
   public AclStatus getAclStatus(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getAclStatus",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (AclStatus) rpcClient.invokeSequential(
-        locations, method, AclStatus.class, null);
+    return clientProto.getAclStatus(src);
   }
 
   @Override // ClientProtocol
   public void createEncryptionZone(String src, String keyName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("createEncryptionZone",
-        new Class<?>[] {String.class, String.class},
-        new RemoteParam(), keyName);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.createEncryptionZone(src, keyName);
   }
 
   @Override // ClientProtocol
   public EncryptionZone getEZForPath(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getEZForPath",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (EncryptionZone) rpcClient.invokeSequential(
-        locations, method, EncryptionZone.class, null);
+    return clientProto.getEZForPath(src);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listEncryptionZones(prevId);
   }
 
   @Override // ClientProtocol
   public void reencryptEncryptionZone(String zone, ReencryptAction action)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.reencryptEncryptionZone(zone, action);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(
       long prevId) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listReencryptionStatus(prevId);
   }
 
   @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("setXAttr",
-        new Class<?>[] {String.class, XAttr.class, EnumSet.class},
-        new RemoteParam(), xAttr, flag);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.setXAttr(src, xAttr, flag);
   }
 
-  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("getXAttrs",
-        new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
-    return (List<XAttr>) rpcClient.invokeSequential(
-        locations, method, List.class, null);
+    return clientProto.getXAttrs(src, xAttrs);
   }
 
-  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> listXAttrs(String src) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, false);
-    RemoteMethod method = new RemoteMethod("listXAttrs",
-        new Class<?>[] {String.class}, new RemoteParam());
-    return (List<XAttr>) rpcClient.invokeSequential(
-        locations, method, List.class, null);
+    return clientProto.listXAttrs(src);
   }
 
   @Override // ClientProtocol
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(src, true);
-    RemoteMethod method = new RemoteMethod("removeXAttr",
-        new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.removeXAttr(src, xAttr);
   }
 
   @Override // ClientProtocol
   public void checkAccess(String path, FsAction mode) throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    // TODO handle virtual directories
-    final List<RemoteLocation> locations = getLocationsForPath(path, true);
-    RemoteMethod method = new RemoteMethod("checkAccess",
-        new Class<?>[] {String.class, FsAction.class},
-        new RemoteParam(), mode);
-    rpcClient.invokeSequential(locations, method);
+    clientProto.checkAccess(path, mode);
   }
 
   @Override // ClientProtocol
   public long getCurrentEditLogTxid() throws IOException {
-    checkOperation(OperationCategory.READ);
-
-    RemoteMethod method = new RemoteMethod(
-        "getCurrentEditLogTxid", new Class<?>[] {});
-    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Long> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
-
-    // Return the maximum txid
-    long txid = 0;
-    for (long t : ret.values()) {
-      if (t > txid) {
-        txid = t;
-      }
-    }
-    return txid;
+    return clientProto.getCurrentEditLogTxid();
   }
 
   @Override // ClientProtocol
   public EventBatchList getEditsFromTxid(long txid) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getEditsFromTxid(txid);
   }
 
-  @Override
+  @Override // ClientProtocol
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getDataEncryptionKey();
   }
 
-  @Override
+  @Override // ClientProtocol
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE);
-    return null;
+    return clientProto.createSnapshot(snapshotRoot, snapshotName);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.deleteSnapshot(snapshotRoot, snapshotName);
   }
 
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
-    this.quotaCall.setQuota(path, namespaceQuota, storagespaceQuota, type);
+    clientProto.setQuota(path, namespaceQuota, storagespaceQuota, type);
   }
 
   @Override // ClientProtocol
   public QuotaUsage getQuotaUsage(String path) throws IOException {
-    return this.quotaCall.getQuotaUsage(path);
+    return clientProto.getQuotaUsage(path);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    checkOperation(OperationCategory.WRITE);
-
-    // Block pool id -> blocks
-    Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
-    for (LocatedBlock block : blocks) {
-      String bpId = block.getBlock().getBlockPoolId();
-      List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
-      if (bpBlocks == null) {
-        bpBlocks = new LinkedList<>();
-        blockLocations.put(bpId, bpBlocks);
-      }
-      bpBlocks.add(block);
-    }
-
-    // Invoke each block pool
-    for (Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
-      String bpId = entry.getKey();
-      List<LocatedBlock> bpBlocks = entry.getValue();
-
-      LocatedBlock[] bpBlocksArray =
-          bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
-      RemoteMethod method = new RemoteMethod("reportBadBlocks",
-          new Class<?>[] {LocatedBlock[].class},
-          new Object[] {bpBlocksArray});
-      rpcClient.invokeSingleBlockPool(bpId, method);
-    }
+    clientProto.reportBadBlocks(blocks);
   }
 
-  @Override
+  @Override // ClientProtocol
   public void unsetStoragePolicy(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+    clientProto.unsetStoragePolicy(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getStoragePolicy(path);
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
       throws IOException {
-    return erasureCoding.getErasureCodingPolicies();
+    return clientProto.getErasureCodingPolicies();
   }
 
   @Override // ClientProtocol
   public Map<String, String> getErasureCodingCodecs() throws IOException {
-    return erasureCoding.getErasureCodingCodecs();
+    return clientProto.getErasureCodingCodecs();
   }
 
   @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    return erasureCoding.addErasureCodingPolicies(policies);
+    return clientProto.addErasureCodingPolicies(policies);
   }
 
   @Override // ClientProtocol
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
+    clientProto.removeErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void disableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
+    clientProto.disableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void enableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
+    clientProto.enableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws IOException {
-    return erasureCoding.getErasureCodingPolicy(src);
+    return clientProto.getErasureCodingPolicy(src);
   }
 
   @Override // ClientProtocol
   public void setErasureCodingPolicy(String src, String ecPolicyName)
       throws IOException {
-    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+    clientProto.setErasureCodingPolicy(src, ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void unsetErasureCodingPolicy(String src) throws IOException {
-    erasureCoding.unsetErasureCodingPolicy(src);
+    clientProto.unsetErasureCodingPolicy(src);
   }
 
-  @Override
+  @Override // ClientProtocol
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    return erasureCoding.getECBlockGroupStats();
+    return clientProto.getECBlockGroupStats();
   }
 
-  @Override
+  @Override // ClientProtocol
   public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.getReplicatedBlockStats();
   }
 
   @Deprecated
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
-        OpenFilesIterator.FILTER_PATH_DEFAULT);
+    return clientProto.listOpenFiles(prevId);
   }
 
-  @Override
+  @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
       EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+    return clientProto.listOpenFiles(prevId, openFilesTypes, path);
+  }
+
+  @Override // ClientProtocol
+  public void satisfyStoragePolicy(String path) throws IOException {
+    clientProto.satisfyStoragePolicy(path);
   }
 
   @Override // NamenodeProtocol
@@ -2167,6 +1266,11 @@ public class RouterRpcServer extends AbstractService
     return nnProto.isRollingUpgrade();
   }
 
+  @Override // NamenodeProtocol
+  public Long getNextSPSPath() throws IOException {
+    return nnProto.getNextSPSPath();
+  }
+
   /**
    * Locate the location with the matching block pool id.
    *
@@ -2176,7 +1280,7 @@ public class RouterRpcServer extends AbstractService
    * @return Prioritized list of locations in the federated cluster.
    * @throws IOException if the location for this path cannot be determined.
    */
-  private RemoteLocation getLocationForPath(
+  protected RemoteLocation getLocationForPath(
       String path, boolean failIfLocked, String blockPoolId)
           throws IOException {
 
@@ -2276,27 +1380,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
-   * Check if a path should be in all subclusters.
-   *
-   * @param path Path to check.
-   * @return If a path should be in all subclusters.
-   */
-  private boolean isPathAll(final String path) {
-    if (subclusterResolver instanceof MountTableResolver) {
-      try {
-        MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
-        MountTable entry = mountTable.getMountPoint(path);
-        if (entry != null) {
-          return entry.isAll();
-        }
-      } catch (IOException e) {
-        LOG.error("Cannot get mount point", e);
-      }
-    }
-    return false;
-  }
-
-  /**
    * Check if a path is in a read only mount point.
    *
    * @param path Path to check.
@@ -2318,121 +1401,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
-   * Get the modification dates for mount points.
-   *
-   * @param path Name of the path to start checking dates from.
-   * @return Map with the modification dates for all sub-entries.
-   */
-  private Map<String, Long> getMountPointDates(String path) {
-    Map<String, Long> ret = new TreeMap<>();
-    if (subclusterResolver instanceof MountTableResolver) {
-      try {
-        final List<String> children = subclusterResolver.getMountPoints(path);
-        for (String child : children) {
-          Long modTime = getModifiedTime(ret, path, child);
-          ret.put(child, modTime);
-        }
-      } catch (IOException e) {
-        LOG.error("Cannot get mount point", e);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * Get modified time for child. If the child is present in mount table it
-   * will return the modified time. If the child is not present but subdirs of
-   * this child are present then it will return latest modified subdir's time
-   * as modified time of the requested child.
-   * @param ret contains children and modified times.
-   * @param mountTable.
-   * @param path Name of the path to start checking dates from.
-   * @param child child of the requested path.
-   * @return modified time.
-   */
-  private long getModifiedTime(Map<String, Long> ret, String path,
-      String child) {
-    MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
-    String srcPath;
-    if (path.equals(Path.SEPARATOR)) {
-      srcPath = Path.SEPARATOR + child;
-    } else {
-      srcPath = path + Path.SEPARATOR + child;
-    }
-    Long modTime = 0L;
-    try {
-      // Get mount table entry for the srcPath
-      MountTable entry = mountTable.getMountPoint(srcPath);
-      // if srcPath is not in mount table but its subdirs are in mount
-      // table we will display latest modified subdir date/time.
-      if (entry == null) {
-        List<MountTable> entries = mountTable.getMounts(srcPath);
-        for (MountTable eachEntry : entries) {
-          // Get the latest date
-          if (ret.get(child) == null ||
-              ret.get(child) < eachEntry.getDateModified()) {
-            modTime = eachEntry.getDateModified();
-          }
-        }
-      } else {
-        modTime = entry.getDateModified();
-      }
-    } catch (IOException e) {
-      LOG.error("Cannot get mount point", e);
-    }
-    return modTime;
-  }
-
-  /**
-   * Create a new file status for a mount point.
-   *
-   * @param name Name of the mount point.
-   * @param childrenNum Number of children.
-   * @param date Map with the dates.
-   * @return New HDFS file status representing a mount point.
-   */
-  private HdfsFileStatus getMountPointStatus(
-      String name, int childrenNum, long date) {
-    long modTime = date;
-    long accessTime = date;
-    FsPermission permission = FsPermission.getDirDefault();
-    String owner = this.superUser;
-    String group = this.superGroup;
-    try {
-      // TODO support users, it should be the user for the pointed folder
-      UserGroupInformation ugi = getRemoteUser();
-      owner = ugi.getUserName();
-      group = ugi.getPrimaryGroupName();
-    } catch (IOException e) {
-      LOG.error("Cannot get the remote user: {}", e.getMessage());
-    }
-    long inodeId = 0;
-    return new HdfsFileStatus.Builder()
-      .isdir(true)
-      .mtime(modTime)
-      .atime(accessTime)
-      .perm(permission)
-      .owner(owner)
-      .group(group)
-      .symlink(new byte[0])
-      .path(DFSUtil.string2Bytes(name))
-      .fileId(inodeId)
-      .children(childrenNum)
-      .build();
-  }
-
-  /**
-   * Get the name of the method that is calling this function.
-   *
-   * @return Name of the method calling this function.
-   */
-  private static String getMethodName() {
-    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
-    String methodName = stack[3].getMethodName();
-    return methodName;
-  }
-
-  /**
    * Get the user that is invoking this operation.
    *
    * @return Remote user group information.
@@ -2490,16 +1458,4 @@ public class RouterRpcServer extends AbstractService
   public FederationRPCMetrics getRPCMetrics() {
     return this.rpcMonitor.getRPCMetrics();
   }
-
-  @Override
-  public void satisfyStoragePolicy(String path) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
-  @Override
-  public Long getNextSPSPath() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    // not supported
-    return null;
-  }
 }


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


[34/50] [abbrv] hadoop git commit: HDDS-356. Support ColumnFamily based RockDBStore and TableStore. Contributed by Anu Engineer.

Posted by sh...@apache.org.
HDDS-356. Support ColumnFamily based RockDBStore and TableStore.
Contributed by Anu Engineer.


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

Branch: refs/heads/HDFS-12943
Commit: b021249ac84abe31c9d30d73ed483bea2acdbaab
Parents: af4b705
Author: Anu Engineer <ae...@apache.org>
Authored: Wed Aug 22 18:55:14 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Wed Aug 22 18:55:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/utils/db/DBStore.java     |  93 +++++++
 .../org/apache/hadoop/utils/db/RDBStore.java    | 252 +++++++++++++++++++
 .../hadoop/utils/db/RDBStoreIterator.java       |  88 +++++++
 .../org/apache/hadoop/utils/db/RDBTable.java    | 173 +++++++++++++
 .../java/org/apache/hadoop/utils/db/Table.java  | 150 +++++++++++
 .../apache/hadoop/utils/db/TableIterator.java   |  50 ++++
 .../apache/hadoop/utils/db/package-info.java    |  22 ++
 .../apache/hadoop/utils/db/TestRDBStore.java    | 246 ++++++++++++++++++
 .../hadoop/utils/db/TestRDBTableStore.java      | 189 ++++++++++++++
 .../apache/hadoop/utils/db/package-info.java    |  22 ++
 10 files changed, 1285 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
new file mode 100644
index 0000000..a817f4f
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
@@ -0,0 +1,93 @@
+/*
+ * 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.utils.db;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+/**
+ * The DBStore interface provides the ability to create Tables, which store
+ * a specific type of Key-Value pair. Some DB interfaces like LevelDB will not
+ * be able to do this. In those case a Table creation will map to a default
+ * store.
+ *
+ */
+@InterfaceStability.Evolving
+public interface DBStore extends AutoCloseable {
+
+  /**
+   * Gets an existing TableStore.
+   *
+   * @param name - Name of the TableStore to get
+   * @return - TableStore.
+   * @throws IOException on Failure
+   */
+  Table getTable(String name) throws IOException;
+
+  /**
+   * Lists the Known list of Tables in a DB.
+   *
+   * @return List of Tables, in case of Rocks DB and LevelDB we will return at
+   * least one entry called DEFAULT.
+   * @throws IOException on Failure
+   */
+  ArrayList<Table> listTables() throws IOException;
+
+  /**
+   * Compact the entire database.
+   *
+   * @throws IOException on Failure
+   */
+  void compactDB() throws IOException;
+
+  /**
+   * Moves a key from the Source Table to the destination Table.
+   *
+   * @param key - Key to move.
+   * @param source - Source Table.
+   * @param dest - Destination Table.
+   * @throws IOException on Failure
+   */
+  void move(byte[] key, Table source, Table dest) throws IOException;
+
+  /**
+   * Moves a key from the Source Table to the destination Table and updates the
+   * destination to the new value.
+   *
+   * @param key - Key to move.
+   * @param value - new value to write to the destination table.
+   * @param source - Source Table.
+   * @param dest - Destination Table.
+   * @throws IOException on Failure
+   */
+  void move(byte[] key, byte[] value, Table source, Table dest)
+      throws IOException;
+
+  /**
+   * Returns an estimated count of keys in this DB.
+   *
+   * @return long, estimate of keys in the DB.
+   */
+  long getEstimatedKeyCount() throws IOException;
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
new file mode 100644
index 0000000..c719d31
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
@@ -0,0 +1,252 @@
+/*
+ * 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.utils.db;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.utils.RocksDBStoreMBean;
+import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.ObjectName;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * RocksDB Store that supports creating Tables in DB.
+ */
+public class RDBStore implements DBStore {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RDBStore.class);
+  private final RocksDB db;
+  private final File dbLocation;
+  private final WriteOptions writeOptions;
+  private final DBOptions dbOptions;
+  private final Hashtable<String, ColumnFamilyHandle> handleTable;
+  private ObjectName statMBeanName;
+
+  public RDBStore(File dbFile, DBOptions options, List<String> families)
+      throws IOException {
+    Preconditions.checkNotNull(dbFile, "DB file location cannot be null");
+    Preconditions.checkNotNull(families);
+    Preconditions.checkArgument(families.size() > 0);
+    handleTable = new Hashtable<>();
+
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    dbOptions = options;
+    dbLocation = dbFile;
+    // TODO: Read from the next Config.
+    writeOptions = new WriteOptions();
+
+    try {
+      db = RocksDB.open(dbOptions, dbLocation.getAbsolutePath(),
+          columnFamilyDescriptors, columnFamilyHandles);
+
+      for (int x = 0; x < columnFamilyHandles.size(); x++) {
+        handleTable.put(
+            DFSUtil.bytes2String(columnFamilyHandles.get(x).getName()),
+            columnFamilyHandles.get(x));
+      }
+
+      if (dbOptions.statistics() != null) {
+        Map<String, String> jmxProperties = new HashMap<>();
+        jmxProperties.put("dbName", dbFile.getName());
+        statMBeanName = MBeans.register("Ozone", "RocksDbStore", jmxProperties,
+            new RocksDBStoreMBean(dbOptions.statistics()));
+        if (statMBeanName == null) {
+          LOG.warn("jmx registration failed during RocksDB init, db path :{}",
+              dbFile.getAbsolutePath());
+        }
+      }
+
+    } catch (RocksDBException e) {
+      throw toIOException(
+          "Failed init RocksDB, db path : " + dbFile.getAbsolutePath(), e);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("RocksDB successfully opened.");
+      LOG.debug("[Option] dbLocation= {}", dbLocation.getAbsolutePath());
+      LOG.debug("[Option] createIfMissing = {}", options.createIfMissing());
+      LOG.debug("[Option] maxOpenFiles= {}", options.maxOpenFiles());
+    }
+  }
+
+  public static IOException toIOException(String msg, RocksDBException e) {
+    String statusCode = e.getStatus() == null ? "N/A" :
+        e.getStatus().getCodeString();
+    String errMessage = e.getMessage() == null ? "Unknown error" :
+        e.getMessage();
+    String output = msg + "; status : " + statusCode
+        + "; message : " + errMessage;
+    return new IOException(output, e);
+  }
+
+  @Override
+  public void compactDB() throws IOException {
+    if (db != null) {
+      try {
+        db.compactRange();
+      } catch (RocksDBException e) {
+        throw toIOException("Failed to compact db", e);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+    for (final ColumnFamilyHandle handle : handleTable.values()) {
+      handle.close();
+    }
+    if (dbOptions != null) {
+      dbOptions.close();
+    }
+    if (writeOptions != null) {
+      writeOptions.close();
+    }
+    if (statMBeanName != null) {
+      MBeans.unregister(statMBeanName);
+      statMBeanName = null;
+    }
+    if (db != null) {
+      db.close();
+    }
+  }
+
+  @Override
+  public void move(byte[] key, Table source, Table dest) throws IOException {
+    RDBTable sourceTable;
+    RDBTable destTable;
+    if (source instanceof RDBTable) {
+      sourceTable = (RDBTable) source;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Source.");
+      throw new IOException("Unexpected TableStore Type in source. Expected "
+          + "RocksDBTable.");
+    }
+
+    if (dest instanceof RDBTable) {
+      destTable = (RDBTable) dest;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Dest.");
+      throw new IOException("Unexpected TableStore Type in dest. Expected "
+          + "RocksDBTable.");
+    }
+    try (WriteBatch batch = new WriteBatch()) {
+      byte[] value = sourceTable.get(key);
+      batch.put(destTable.getHandle(), key, value);
+      batch.delete(sourceTable.getHandle(), key);
+      db.write(writeOptions, batch);
+    } catch (RocksDBException rockdbException) {
+      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(key));
+      throw toIOException("Unable to move key: " + DFSUtil.bytes2String(key),
+          rockdbException);
+    }
+  }
+
+  @Override
+  public void move(byte[] key, byte[] value, Table source,
+      Table dest) throws IOException {
+    RDBTable sourceTable;
+    RDBTable destTable;
+    if (source instanceof RDBTable) {
+      sourceTable = (RDBTable) source;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Source.");
+      throw new IOException("Unexpected TableStore Type in source. Expected "
+          + "RocksDBTable.");
+    }
+
+    if (dest instanceof RDBTable) {
+      destTable = (RDBTable) dest;
+    } else {
+      LOG.error("Unexpected Table type. Expected RocksTable Store for Dest.");
+      throw new IOException("Unexpected TableStore Type in dest. Expected "
+          + "RocksDBTable.");
+    }
+    try (WriteBatch batch = new WriteBatch()) {
+      batch.put(destTable.getHandle(), key, value);
+      batch.delete(sourceTable.getHandle(), key);
+      db.write(writeOptions, batch);
+    } catch (RocksDBException rockdbException) {
+      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(key));
+      throw toIOException("Unable to move key: " + DFSUtil.bytes2String(key),
+          rockdbException);
+    }
+  }
+
+  @Override
+  public long getEstimatedKeyCount() throws IOException {
+    try {
+      return Long.parseLong(db.getProperty("rocksdb.estimate-num-keys"));
+    } catch (RocksDBException e) {
+      throw toIOException("Unable to get the estimated count.", e);
+    }
+  }
+
+  @VisibleForTesting
+  protected ObjectName getStatMBeanName() {
+    return statMBeanName;
+  }
+
+  @Override
+  public Table getTable(String name) throws IOException {
+    ColumnFamilyHandle handle = handleTable.get(name);
+    if (handle == null) {
+      throw new IOException("No such table in this DB. TableName : " + name);
+    }
+    return new RDBTable(this.db, handle, this.writeOptions);
+  }
+
+  @Override
+  public ArrayList<Table> listTables() throws IOException {
+    ArrayList<Table> returnList = new ArrayList<>();
+    for (ColumnFamilyHandle handle: handleTable.values())  {
+      returnList.add(new RDBTable(db, handle, writeOptions));
+    }
+    return returnList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
new file mode 100644
index 0000000..f1f2df6
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStoreIterator.java
@@ -0,0 +1,88 @@
+/*
+ * 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.utils.db;
+
+import org.apache.hadoop.utils.db.Table.KeyValue;
+import org.rocksdb.RocksIterator;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.function.Consumer;
+
+/**
+ * RocksDB store iterator.
+ */
+public class RDBStoreIterator implements TableIterator<KeyValue> {
+
+  private RocksIterator rocksDBIterator;
+
+  public RDBStoreIterator(RocksIterator iterator) {
+    this.rocksDBIterator = iterator;
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void forEachRemaining(Consumer<? super KeyValue> action) {
+    while(hasNext()) {
+      action.accept(next());
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    return rocksDBIterator.isValid();
+  }
+
+  @Override
+  public Table.KeyValue next() {
+    if (rocksDBIterator.isValid()) {
+      KeyValue value = KeyValue.create(rocksDBIterator.key(), rocksDBIterator
+          .value());
+      rocksDBIterator.next();
+      return value;
+    }
+    throw new NoSuchElementException("RocksDB Store has no more elements");
+  }
+
+  @Override
+  public void seekToFirst() {
+    rocksDBIterator.seekToFirst();
+  }
+
+  @Override
+  public void seekToLast() {
+    rocksDBIterator.seekToLast();
+  }
+
+  @Override
+  public KeyValue seek(byte[] key) {
+    rocksDBIterator.seek(key);
+    if (rocksDBIterator.isValid()) {
+      return KeyValue.create(rocksDBIterator.key(),
+          rocksDBIterator.value());
+    }
+    return null;
+  }
+
+  @Override
+  public void close() throws IOException {
+    rocksDBIterator.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
new file mode 100644
index 0000000..8cf6b35
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBTable.java
@@ -0,0 +1,173 @@
+/*
+ * 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.utils.db;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * RocksDB implementation of ozone metadata store.
+ */
+public class RDBTable implements Table {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RDBTable.class);
+
+  private final RocksDB db;
+  private final ColumnFamilyHandle handle;
+  private final WriteOptions writeOptions;
+
+  /**
+   * Constructs a TableStore.
+   *
+   * @param db - DBstore that we are using.
+   * @param handle - ColumnFamily Handle.
+   * @param writeOptions - RocksDB write Options.
+   */
+  public RDBTable(RocksDB db, ColumnFamilyHandle handle,
+      WriteOptions writeOptions) {
+    this.db = db;
+    this.handle = handle;
+    this.writeOptions = writeOptions;
+  }
+
+  /**
+   * Converts RocksDB exception to IOE.
+   * @param msg  - Message to add to exception.
+   * @param e - Original Exception.
+   * @return  IOE.
+   */
+  public static IOException toIOException(String msg, RocksDBException e) {
+    String statusCode = e.getStatus() == null ? "N/A" :
+        e.getStatus().getCodeString();
+    String errMessage = e.getMessage() == null ? "Unknown error" :
+        e.getMessage();
+    String output = msg + "; status : " + statusCode
+        + "; message : " + errMessage;
+    return new IOException(output, e);
+  }
+
+  /**
+   * Returns the Column family Handle.
+   *
+   * @return ColumnFamilyHandle.
+   */
+  @Override
+  public ColumnFamilyHandle getHandle() {
+    return handle;
+  }
+
+  @Override
+  public void put(byte[] key, byte[] value) throws IOException {
+    try {
+      db.put(handle, writeOptions, key, value);
+    } catch (RocksDBException e) {
+      LOG.error("Failed to write to DB. Key: {}", new String(key,
+          StandardCharsets.UTF_8));
+      throw toIOException("Failed to put key-value to metadata "
+          + "store", e);
+    }
+  }
+
+  @Override
+  public boolean isEmpty() throws IOException {
+    try (TableIterator<KeyValue> keyIter = iterator()) {
+      keyIter.seekToFirst();
+      return !keyIter.hasNext();
+    }
+  }
+
+  @Override
+  public byte[] get(byte[] key) throws IOException {
+    try {
+      return db.get(handle, key);
+    } catch (RocksDBException e) {
+      throw toIOException(
+          "Failed to get the value for the given key", e);
+    }
+  }
+
+  @Override
+  public void delete(byte[] key) throws IOException {
+    try {
+      db.delete(handle, key);
+    } catch (RocksDBException e) {
+      throw toIOException("Failed to delete the given key", e);
+    }
+  }
+
+  @Override
+  public void writeBatch(WriteBatch operation) throws IOException {
+    try {
+      db.write(writeOptions, operation);
+    } catch (RocksDBException e) {
+      throw toIOException("Batch write operation failed", e);
+    }
+  }
+
+//  @Override
+//  public void iterate(byte[] from, EntryConsumer consumer)
+//      throws IOException {
+//
+//    try (RocksIterator it = db.newIterator(handle)) {
+//      if (from != null) {
+//        it.seek(from);
+//      } else {
+//        it.seekToFirst();
+//      }
+//      while (it.isValid()) {
+//        if (!consumer.consume(it.key(), it.value())) {
+//          break;
+//        }
+//        it.next();
+//      }
+//    }
+//  }
+
+  @Override
+  public TableIterator<KeyValue> iterator() {
+    ReadOptions readOptions = new ReadOptions();
+    return new RDBStoreIterator(db.newIterator(handle, readOptions));
+  }
+
+  @Override
+  public String getName() throws IOException {
+    try {
+      return DFSUtil.bytes2String(this.getHandle().getName());
+    } catch (RocksDBException rdbEx) {
+      throw toIOException("Unable to get the table name.", rdbEx);
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    // Nothing do for a Column Family.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
new file mode 100644
index 0000000..3942585
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/Table.java
@@ -0,0 +1,150 @@
+/*
+ * 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.utils.db;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.WriteBatch;
+
+import java.io.IOException;
+
+/**
+ * Interface for key-value store that stores ozone metadata. Ozone metadata is
+ * stored as key value pairs, both key and value are arbitrary byte arrays. Each
+ * Table Stores a certain kind of keys and values. This allows a DB to have
+ * different kind of tables.
+ */
+@InterfaceStability.Evolving
+public interface Table extends AutoCloseable {
+
+  /**
+   * Puts a key-value pair into the store.
+   *
+   * @param key metadata key
+   * @param value metadata value
+   */
+  void put(byte[] key, byte[] value) throws IOException;
+
+  /**
+   * @return true if the metadata store is empty.
+   * @throws IOException on Failure
+   */
+  boolean isEmpty() throws IOException;
+
+  /**
+   * Returns the value mapped to the given key in byte array or returns null
+   * if the key is not found.
+   *
+   * @param key metadata key
+   * @return value in byte array or null if the key is not found.
+   * @throws IOException on Failure
+   */
+  byte[] get(byte[] key) throws IOException;
+
+  /**
+   * Deletes a key from the metadata store.
+   *
+   * @param key metadata key
+   * @throws IOException on Failure
+   */
+  void delete(byte[] key) throws IOException;
+
+  /**
+   * Return the Column Family handle. TODO: This leaks an RockDB abstraction
+   * into Ozone code, cleanup later.
+   *
+   * @return ColumnFamilyHandle
+   */
+  ColumnFamilyHandle getHandle();
+
+  /**
+   * A batch of PUT, DELETE operations handled as a single atomic write.
+   *
+   * @throws IOException write fails
+   */
+  void writeBatch(WriteBatch operation) throws IOException;
+
+  /**
+   * Returns the iterator for this metadata store.
+   *
+   * @return MetaStoreIterator
+   */
+  TableIterator<KeyValue> iterator();
+
+  /**
+   * Returns the Name of this Table.
+   * @return - Table Name.
+   * @throws IOException on failure.
+   */
+  String getName() throws IOException;
+
+  /**
+   * Class used to represent the key and value pair of a db entry.
+   */
+  class KeyValue {
+
+    private final byte[] key;
+    private final byte[] value;
+
+    /**
+     * KeyValue Constructor, used to represent a key and value of a db entry.
+     *
+     * @param key - Key Bytes
+     * @param value - Value bytes
+     */
+    private KeyValue(byte[] key, byte[] value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    /**
+     * Create a KeyValue pair.
+     *
+     * @param key - Key Bytes
+     * @param value - Value bytes
+     * @return KeyValue object.
+     */
+    public static KeyValue create(byte[] key, byte[] value) {
+      return new KeyValue(key, value);
+    }
+
+    /**
+     * Return key.
+     *
+     * @return byte[]
+     */
+    public byte[] getKey() {
+      byte[] result = new byte[key.length];
+      System.arraycopy(key, 0, result, 0, key.length);
+      return result;
+    }
+
+    /**
+     * Return value.
+     *
+     * @return byte[]
+     */
+    public byte[] getValue() {
+      byte[] result = new byte[value.length];
+      System.arraycopy(value, 0, result, 0, value.length);
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
new file mode 100644
index 0000000..83a8f3c
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableIterator.java
@@ -0,0 +1,50 @@
+/*
+ * 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.utils.db;
+
+import java.io.Closeable;
+import java.util.Iterator;
+
+/**
+ * Iterator for MetaDataStore DB.
+ *
+ * @param <T>
+ */
+public interface TableIterator<T> extends Iterator<T>, Closeable {
+
+  /**
+   * seek to first entry.
+   */
+  void seekToFirst();
+
+  /**
+   * seek to last entry.
+   */
+  void seekToLast();
+
+  /**
+   * Seek to the specific key.
+   *
+   * @param key - Bytes that represent the key.
+   * @return T.
+   */
+  T seek(byte[] key);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
new file mode 100644
index 0000000..17d676d
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Database interfaces for Ozone.
+ */
+package org.apache.hadoop.utils.db;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
new file mode 100644
index 0000000..94a650b
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
@@ -0,0 +1,246 @@
+/*
+ * 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.utils.db;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.Statistics;
+import org.rocksdb.StatsLevel;
+
+import javax.management.MBeanServer;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * RDBStore Tests.
+ */
+public class TestRDBStore {
+  private final List<String> families =
+      Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+          "First", "Second", "Third",
+          "Fourth", "Fifth",
+          "Sixth");
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  private RDBStore rdbStore = null;
+  private DBOptions options = null;
+
+  @Before
+  public void setUp() throws Exception {
+    options = new DBOptions();
+    options.setCreateIfMissing(true);
+    options.setCreateMissingColumnFamilies(true);
+
+    Statistics statistics = new Statistics();
+    statistics.setStatsLevel(StatsLevel.ALL);
+    options = options.setStatistics(statistics);
+    rdbStore = new RDBStore(folder.newFolder(), options, families);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (rdbStore != null) {
+      rdbStore.close();
+    }
+  }
+
+  @Test
+  public void compactDB() throws Exception {
+    try (RDBStore newStore =
+             new RDBStore(folder.newFolder(), options, families)) {
+      Assert.assertNotNull("DB Store cannot be null", newStore);
+      try (Table firstTable = newStore.getTable(families.get(1))) {
+        Assert.assertNotNull("Table cannot be null", firstTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          firstTable.put(key, value);
+        }
+      }
+      // This test does not assert anything if there is any error this test
+      // will throw and fail.
+      newStore.compactDB();
+    }
+  }
+
+  @Test
+  public void close() throws Exception {
+    RDBStore newStore =
+        new RDBStore(folder.newFolder(), options, families);
+    Assert.assertNotNull("DBStore cannot be null", newStore);
+    // This test does not assert anything if there is any error this test
+    // will throw and fail.
+    newStore.close();
+  }
+
+  @Test
+  public void moveKey() throws Exception {
+    byte[] key =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      firstTable.put(key, value);
+      try (Table secondTable = rdbStore.getTable(families.get(2))) {
+        rdbStore.move(key, firstTable, secondTable);
+        byte[] newvalue = secondTable.get(key);
+        // Make sure we have value in the second table
+        Assert.assertNotNull(newvalue);
+        //and it is same as what we wrote to the FirstTable
+        Assert.assertArrayEquals(value, newvalue);
+      }
+      // After move this key must not exist in the first table.
+      Assert.assertNull(firstTable.get(key));
+    }
+  }
+
+  @Test
+  public void moveWithValue() throws Exception {
+    byte[] key =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+
+    byte[] nextValue =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      firstTable.put(key, value);
+      try (Table secondTable = rdbStore.getTable(families.get(2))) {
+        rdbStore.move(key, nextValue, firstTable, secondTable);
+        byte[] newvalue = secondTable.get(key);
+        // Make sure we have value in the second table
+        Assert.assertNotNull(newvalue);
+        //and it is not same as what we wrote to the FirstTable, and equals
+        // the new value.
+        Assert.assertArrayEquals(nextValue, nextValue);
+      }
+    }
+
+  }
+
+  @Test
+  public void getEstimatedKeyCount() throws Exception {
+    try (RDBStore newStore =
+             new RDBStore(folder.newFolder(), options, families)) {
+      Assert.assertNotNull("DB Store cannot be null", newStore);
+      // Write 100 keys to the first table.
+      try (Table firstTable = newStore.getTable(families.get(1))) {
+        Assert.assertNotNull("Table cannot be null", firstTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          firstTable.put(key, value);
+        }
+      }
+
+      // Write 100 keys to the secondTable table.
+      try (Table secondTable = newStore.getTable(families.get(2))) {
+        Assert.assertNotNull("Table cannot be null", secondTable);
+        for (int x = 0; x < 100; x++) {
+          byte[] key =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          byte[] value =
+              RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+          secondTable.put(key, value);
+        }
+      }
+      // Let us make sure that our estimate is not off by 10%
+      Assert.assertTrue(newStore.getEstimatedKeyCount() > 180
+          || newStore.getEstimatedKeyCount() < 220);
+    }
+  }
+
+  @Test
+  public void getStatMBeanName() throws Exception {
+
+    try (Table firstTable = rdbStore.getTable(families.get(1))) {
+      for (int y = 0; y < 100; y++) {
+        byte[] key =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+      }
+    }
+    MBeanServer platformMBeanServer =
+        ManagementFactory.getPlatformMBeanServer();
+    Thread.sleep(2000);
+
+    Object keysWritten = platformMBeanServer
+        .getAttribute(rdbStore.getStatMBeanName(), "NUMBER_KEYS_WRITTEN");
+
+    Assert.assertTrue(((Long) keysWritten) >= 99L);
+
+    Object dbWriteAverage = platformMBeanServer
+        .getAttribute(rdbStore.getStatMBeanName(), "DB_WRITE_AVERAGE");
+    Assert.assertTrue((double) dbWriteAverage > 0);
+  }
+
+  @Test
+  public void getTable() throws Exception {
+    for (String tableName : families) {
+      try (Table table = rdbStore.getTable(tableName)) {
+        Assert.assertNotNull(tableName + "is null", table);
+      }
+    }
+    thrown.expect(IOException.class);
+    rdbStore.getTable("ATableWithNoName");
+  }
+
+  @Test
+  public void listTables() throws Exception {
+    List<Table> tableList = rdbStore.listTables();
+    Assert.assertNotNull("Table list cannot be null", tableList);
+    Map<String, Table> hashTable = new HashMap<>();
+
+    for (Table t : tableList) {
+      hashTable.put(t.getName(), t);
+    }
+
+    int count = families.size();
+    // Assert that we have all the tables in the list and no more.
+    for (String name : families) {
+      Assert.assertTrue(hashTable.containsKey(name));
+      count--;
+    }
+    Assert.assertEquals(0, count);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
new file mode 100644
index 0000000..677a1f9
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
@@ -0,0 +1,189 @@
+/*
+ * 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.utils.db;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.Statistics;
+import org.rocksdb.StatsLevel;
+import org.rocksdb.WriteBatch;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Tests for RocksDBTable Store.
+ */
+public class TestRDBTableStore {
+  private static int count = 0;
+  private final List<String> families =
+      Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+          "First", "Second", "Third",
+          "Fourth", "Fifth",
+          "Sixth");
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private RDBStore rdbStore = null;
+  private DBOptions options = null;
+
+  @Before
+  public void setUp() throws Exception {
+    options = new DBOptions();
+    options.setCreateIfMissing(true);
+    options.setCreateMissingColumnFamilies(true);
+
+    Statistics statistics = new Statistics();
+    statistics.setStatsLevel(StatsLevel.ALL);
+    options = options.setStatistics(statistics);
+    rdbStore = new RDBStore(folder.newFolder(), options, families);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (rdbStore != null) {
+      rdbStore.close();
+    }
+  }
+
+  @Test
+  public void toIOException() {
+  }
+
+  @Test
+  public void getHandle() throws Exception {
+    try (Table testTable = rdbStore.getTable("First")) {
+      Assert.assertNotNull(testTable);
+      Assert.assertNotNull(testTable.getHandle());
+    }
+  }
+
+  @Test
+  public void putGetAndEmpty() throws Exception {
+    try (Table testTable = rdbStore.getTable("First")) {
+      byte[] key =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      byte[] value =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      testTable.put(key, value);
+      Assert.assertFalse(testTable.isEmpty());
+      byte[] readValue = testTable.get(key);
+      Assert.assertArrayEquals(value, readValue);
+    }
+    try (Table secondTable = rdbStore.getTable("Second")) {
+      Assert.assertTrue(secondTable.isEmpty());
+    }
+  }
+
+  @Test
+  public void delete() throws Exception {
+    List<byte[]> deletedKeys = new LinkedList<>();
+    List<byte[]> validKeys = new LinkedList<>();
+    byte[] value =
+        RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+    for (int x = 0; x < 100; x++) {
+      deletedKeys.add(
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8));
+    }
+
+    for (int x = 0; x < 100; x++) {
+      validKeys.add(
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8));
+    }
+
+    // Write all the keys and delete the keys scheduled for delete.
+    //Assert we find only expected keys in the Table.
+    try (Table testTable = rdbStore.getTable("Fourth")) {
+      for (int x = 0; x < deletedKeys.size(); x++) {
+        testTable.put(deletedKeys.get(x), value);
+        testTable.delete(deletedKeys.get(x));
+      }
+
+      for (int x = 0; x < validKeys.size(); x++) {
+        testTable.put(validKeys.get(x), value);
+      }
+
+      for (int x = 0; x < validKeys.size(); x++) {
+        Assert.assertNotNull(testTable.get(validKeys.get(0)));
+      }
+
+      for (int x = 0; x < deletedKeys.size(); x++) {
+        Assert.assertNull(testTable.get(deletedKeys.get(0)));
+      }
+    }
+  }
+
+  @Test
+  public void writeBatch() throws Exception {
+    WriteBatch batch = new WriteBatch();
+    try (Table testTable = rdbStore.getTable("Fifth")) {
+      byte[] key =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      byte[] value =
+          RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+      batch.put(testTable.getHandle(), key, value);
+      testTable.writeBatch(batch);
+      Assert.assertNotNull(testTable.get(key));
+    }
+    batch.close();
+  }
+
+  private static boolean consume(Table.KeyValue keyValue) {
+    count++;
+    Assert.assertNotNull(keyValue.getKey());
+    return true;
+  }
+
+  @Test
+  public void forEachAndIterator() throws Exception {
+    final int iterCount = 100;
+    try (Table testTable = rdbStore.getTable("Sixth")) {
+      for (int x = 0; x < iterCount; x++) {
+        byte[] key =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(10).getBytes(StandardCharsets.UTF_8);
+        testTable.put(key, value);
+      }
+      int localCount = 0;
+      try (TableIterator<Table.KeyValue> iter = testTable.iterator()) {
+        while (iter.hasNext()) {
+          Table.KeyValue keyValue = iter.next();
+          localCount++;
+        }
+
+        Assert.assertEquals(iterCount, localCount);
+        iter.seekToFirst();
+        iter.forEachRemaining(TestRDBTableStore::consume);
+        Assert.assertEquals(iterCount, count);
+
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b021249a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
new file mode 100644
index 0000000..f06855e
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Tests for the DB Utilities.
+ */
+package org.apache.hadoop.utils.db;
\ No newline at end of file


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


[07/50] [abbrv] hadoop git commit: Revert "HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun."

Posted by sh...@apache.org.
Revert "HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun."

This reverts commit fa121eb66bc42e9cb5586f8c2e268cfdc2ed187a.


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

Branch: refs/heads/HDFS-12943
Commit: fb5b3dce6192265bce9b9d93ab663bdc5be8048e
Parents: 9dd5d5b
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Fri Aug 17 08:01:44 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Fri Aug 17 08:01:44 2018 -0700

----------------------------------------------------------------------
 .../federation/router/RouterRpcServer.java      | 1360 ++++++++++++++++--
 1 file changed, 1202 insertions(+), 158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb5b3dce/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index fe54993..29f32a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -33,12 +33,16 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
@@ -50,6 +54,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -59,6 +64,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
@@ -87,6 +93,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -94,8 +101,8 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
@@ -160,6 +167,11 @@ public class RouterRpcServer extends AbstractService
   /** Configuration for the RPC server. */
   private Configuration conf;
 
+  /** Identifier for the super user. */
+  private final String superUser;
+  /** Identifier for the super group. */
+  private final String superGroup;
+
   /** Router using this RPC server. */
   private final Router router;
 
@@ -187,10 +199,11 @@ public class RouterRpcServer extends AbstractService
   // Modules implementing groups of RPC calls
   /** Router Quota calls. */
   private final Quota quotaCall;
+  /** Erasure coding calls. */
+  private final ErasureCoding erasureCoding;
   /** NamenodeProtocol calls. */
   private final RouterNamenodeProtocol nnProto;
-  /** ClientProtocol calls. */
-  private final RouterClientProtocol clientProto;
+
 
   /**
    * Construct a router RPC server.
@@ -210,6 +223,12 @@ public class RouterRpcServer extends AbstractService
     this.namenodeResolver = nnResolver;
     this.subclusterResolver = fileResolver;
 
+    // User and group for reporting
+    this.superUser = System.getProperty("user.name");
+    this.superGroup = this.conf.get(
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+
     // RPC server settings
     int handlerCount = this.conf.getInt(DFS_ROUTER_HANDLER_COUNT_KEY,
         DFS_ROUTER_HANDLER_COUNT_DEFAULT);
@@ -296,8 +315,8 @@ public class RouterRpcServer extends AbstractService
 
     // Initialize modules
     this.quotaCall = new Quota(this.router, this);
+    this.erasureCoding = new ErasureCoding(this);
     this.nnProto = new RouterNamenodeProtocol(this);
-    this.clientProto = new RouterClientProtocol(conf, this);
   }
 
   @Override
@@ -352,13 +371,6 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
-   * Get the active namenode resolver
-   */
-  public ActiveNamenodeResolver getNamenodeResolver() {
-    return namenodeResolver;
-  }
-
-  /**
    * Get the RPC monitor and metrics.
    *
    * @return RPC monitor and metrics.
@@ -399,7 +411,7 @@ public class RouterRpcServer extends AbstractService
    *                           client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  void checkOperation(OperationCategory op, boolean supported)
+  protected void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -421,7 +433,7 @@ public class RouterRpcServer extends AbstractService
    * @throws SafeModeException If the Router is in safe mode and cannot serve
    *                           client requests.
    */
-  void checkOperation(OperationCategory op)
+  protected void checkOperation(OperationCategory op)
       throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
@@ -452,44 +464,58 @@ public class RouterRpcServer extends AbstractService
     }
   }
 
-  /**
-   * Get the name of the method that is calling this function.
-   *
-   * @return Name of the method calling this function.
-   */
-  static String getMethodName() {
-    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
-    String methodName = stack[3].getMethodName();
-    return methodName;
-  }
-
   @Override // ClientProtocol
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    return clientProto.getDelegationToken(renewer);
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  /**
+   * The the delegation token from each name service.
+   * @param renewer
+   * @return Name service -> Token.
+   * @throws IOException
+   */
+  public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
+      getDelegationTokens(Text renewer) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    return clientProto.renewDelegationToken(token);
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
   }
 
   @Override // ClientProtocol
   public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    clientProto.cancelDelegationToken(token);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public LocatedBlocks getBlockLocations(String src, final long offset,
       final long length) throws IOException {
-    return clientProto.getBlockLocations(src, offset, length);
+    checkOperation(OperationCategory.READ);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), offset, length);
+    return (LocatedBlocks) rpcClient.invokeSequential(locations, remoteMethod,
+        LocatedBlocks.class, null);
   }
 
   @Override // ClientProtocol
   public FsServerDefaults getServerDefaults() throws IOException {
-    return clientProto.getServerDefaults();
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getServerDefaults");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
   }
 
   @Override // ClientProtocol
@@ -498,8 +524,44 @@ public class RouterRpcServer extends AbstractService
       boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
       throws IOException {
-    return clientProto.create(src, masked, clientName, flag, createParent,
+    checkOperation(OperationCategory.WRITE);
+
+    if (createParent && isPathAll(src)) {
+      int index = src.lastIndexOf(Path.SEPARATOR);
+      String parent = src.substring(0, index);
+      LOG.debug("Creating {} requires creating parent {}", src, parent);
+      FsPermission parentPermissions = getParentPermission(masked);
+      boolean success = mkdirs(parent, parentPermissions, createParent);
+      if (!success) {
+        // This shouldn't happen as mkdirs returns true or exception
+        LOG.error("Couldn't create parents for {}", src);
+      }
+    }
+
+    RemoteLocation createLocation = getCreateLocation(src);
+    RemoteMethod method = new RemoteMethod("create",
+        new Class<?>[] {String.class, FsPermission.class, String.class,
+                        EnumSetWritable.class, boolean.class, short.class,
+                        long.class, CryptoProtocolVersion[].class,
+                        String.class},
+        createLocation.getDest(), masked, clientName, flag, createParent,
         replication, blockSize, supportedVersions, ecPolicyName);
+    return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
+  }
+
+  /**
+   * Get the permissions for the parent of a child with given permissions.
+   * Add implicit u+wx permission for parent. This is based on
+   * @{FSDirMkdirOp#addImplicitUwx}.
+   * @param mask The permission mask of the child.
+   * @return The permission mask of the parent.
+   */
+  private static FsPermission getParentPermission(final FsPermission mask) {
+    FsPermission ret = new FsPermission(
+        mask.getUserAction().or(FsAction.WRITE_EXECUTE),
+        mask.getGroupAction(),
+        mask.getOtherAction());
+    return ret;
   }
 
   /**
@@ -510,7 +572,7 @@ public class RouterRpcServer extends AbstractService
    * @return The remote location for this file.
    * @throws IOException If the file has no creation location.
    */
-  RemoteLocation getCreateLocation(final String src)
+  protected RemoteLocation getCreateLocation(final String src)
       throws IOException {
 
     final List<RemoteLocation> locations = getLocationsForPath(src, true);
@@ -551,45 +613,100 @@ public class RouterRpcServer extends AbstractService
     return createLocation;
   }
 
+  // Medium
   @Override // ClientProtocol
   public LastBlockWithStatus append(String src, final String clientName,
       final EnumSetWritable<CreateFlag> flag) throws IOException {
-    return clientProto.append(src, clientName, flag);
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("append",
+        new Class<?>[] {String.class, String.class, EnumSetWritable.class},
+        new RemoteParam(), clientName, flag);
+    return rpcClient.invokeSequential(
+        locations, method, LastBlockWithStatus.class, null);
   }
 
+  // Low
   @Override // ClientProtocol
   public boolean recoverLease(String src, String clientName)
       throws IOException {
-    return clientProto.recoverLease(src, clientName);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("recoverLease",
+        new Class<?>[] {String.class, String.class}, new RemoteParam(),
+        clientName);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
   }
 
   @Override // ClientProtocol
   public boolean setReplication(String src, short replication)
       throws IOException {
-    return clientProto.setReplication(src, replication);
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setReplication",
+        new Class<?>[] {String.class, short.class}, new RemoteParam(),
+        replication);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
   }
 
-  @Override // ClientProtocol
+  @Override
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
-    clientProto.setStoragePolicy(src, policyName);
+    checkOperation(OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setStoragePolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), policyName);
+    rpcClient.invokeSequential(locations, method, null, null);
   }
 
-  @Override // ClientProtocol
+  @Override
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    return clientProto.getStoragePolicies();
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getStoragePolicies");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
   }
 
   @Override // ClientProtocol
   public void setPermission(String src, FsPermission permissions)
       throws IOException {
-    clientProto.setPermission(src, permissions);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setPermission",
+        new Class<?>[] {String.class, FsPermission.class},
+        new RemoteParam(), permissions);
+    if (isPathAll(src)) {
+      rpcClient.invokeConcurrent(locations, method);
+    } else {
+      rpcClient.invokeSequential(locations, method);
+    }
   }
 
   @Override // ClientProtocol
   public void setOwner(String src, String username, String groupname)
       throws IOException {
-    clientProto.setOwner(src, username, groupname);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setOwner",
+        new Class<?>[] {String.class, String.class, String.class},
+        new RemoteParam(), username, groupname);
+    if (isPathAll(src)) {
+      rpcClient.invokeConcurrent(locations, method);
+    } else {
+      rpcClient.invokeSequential(locations, method);
+    }
   }
 
   /**
@@ -601,8 +718,18 @@ public class RouterRpcServer extends AbstractService
       ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
       String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
       throws IOException {
-    return clientProto.addBlock(src, clientName, previous, excludedNodes,
-        fileId, favoredNodes, addBlockFlags);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("addBlock",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+                        DatanodeInfo[].class, long.class, String[].class,
+                        EnumSet.class},
+        new RemoteParam(), clientName, previous, excludedNodes, fileId,
+        favoredNodes, addBlockFlags);
+    // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
+    return (LocatedBlock) rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
   }
 
   /**
@@ -615,26 +742,55 @@ public class RouterRpcServer extends AbstractService
       final String[] existingStorageIDs, final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName)
           throws IOException {
-    return clientProto.getAdditionalDatanode(src, fileId, blk, existings,
-        existingStorageIDs, excludes, numAdditionalNodes, clientName);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
+        new Class<?>[] {String.class, long.class, ExtendedBlock.class,
+                        DatanodeInfo[].class, String[].class,
+                        DatanodeInfo[].class, int.class, String.class},
+        new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
+        numAdditionalNodes, clientName);
+    return (LocatedBlock) rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
   }
 
   @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
       String holder) throws IOException {
-    clientProto.abandonBlock(b, fileId, src, holder);
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("abandonBlock",
+        new Class<?>[] {ExtendedBlock.class, long.class, String.class,
+                        String.class},
+        b, fileId, new RemoteParam(), holder);
+    rpcClient.invokeSingle(b, method);
   }
 
   @Override // ClientProtocol
   public boolean complete(String src, String clientName, ExtendedBlock last,
       long fileId) throws IOException {
-    return clientProto.complete(src, clientName, last, fileId);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("complete",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+                        long.class},
+        new RemoteParam(), clientName, last, fileId);
+    // Complete can return true/false, so don't expect a result
+    return ((Boolean) rpcClient.invokeSequential(
+        locations, method, Boolean.class, null)).booleanValue();
   }
 
   @Override // ClientProtocol
   public LocatedBlock updateBlockForPipeline(
       ExtendedBlock block, String clientName) throws IOException {
-    return clientProto.updateBlockForPipeline(block, clientName);
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
+        new Class<?>[] {ExtendedBlock.class, String.class},
+        block, clientName);
+    return (LocatedBlock) rpcClient.invokeSingle(block, method);
   }
 
   /**
@@ -645,91 +801,462 @@ public class RouterRpcServer extends AbstractService
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
           throws IOException {
-    clientProto.updatePipeline(clientName, oldBlock, newBlock, newNodes,
-        newStorageIDs);
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updatePipeline",
+        new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
+                        DatanodeID[].class, String[].class},
+        clientName, oldBlock, newBlock, newNodes, newStorageIDs);
+    rpcClient.invokeSingle(oldBlock, method);
   }
 
   @Override // ClientProtocol
   public long getPreferredBlockSize(String src) throws IOException {
-    return clientProto.getPreferredBlockSize(src);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return ((Long) rpcClient.invokeSequential(
+        locations, method, Long.class, null)).longValue();
+  }
+
+  /**
+   * Determines combinations of eligible src/dst locations for a rename. A
+   * rename cannot change the namespace. Renames are only allowed if there is an
+   * eligible dst location in the same namespace as the source.
+   *
+   * @param srcLocations List of all potential source destinations where the
+   *          path may be located. On return this list is trimmed to include
+   *          only the paths that have corresponding destinations in the same
+   *          namespace.
+   * @param dst The destination path
+   * @return A map of all eligible source namespaces and their corresponding
+   *         replacement value.
+   * @throws IOException If the dst paths could not be determined.
+   */
+  private RemoteParam getRenameDestinations(
+      final List<RemoteLocation> srcLocations, final String dst)
+          throws IOException {
+
+    final List<RemoteLocation> dstLocations = getLocationsForPath(dst, true);
+    final Map<RemoteLocation, String> dstMap = new HashMap<>();
+
+    Iterator<RemoteLocation> iterator = srcLocations.iterator();
+    while (iterator.hasNext()) {
+      RemoteLocation srcLocation = iterator.next();
+      RemoteLocation eligibleDst =
+          getFirstMatchingLocation(srcLocation, dstLocations);
+      if (eligibleDst != null) {
+        // Use this dst for this source location
+        dstMap.put(srcLocation, eligibleDst.getDest());
+      } else {
+        // This src destination is not valid, remove from the source list
+        iterator.remove();
+      }
+    }
+    return new RemoteParam(dstMap);
+  }
+
+  /**
+   * Get first matching location.
+   *
+   * @param location Location we are looking for.
+   * @param locations List of locations.
+   * @return The first matchin location in the list.
+   */
+  private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
+      List<RemoteLocation> locations) {
+    for (RemoteLocation loc : locations) {
+      if (loc.getNameserviceId().equals(location.getNameserviceId())) {
+        // Return first matching location
+        return loc;
+      }
+    }
+    return null;
   }
 
   @Deprecated
   @Override // ClientProtocol
   public boolean rename(final String src, final String dst)
       throws IOException {
-    return clientProto.rename(src, dst);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations =
+        getLocationsForPath(src, true, false);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+          " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), dstParam);
+    return ((Boolean) rpcClient.invokeSequential(
+        locs, method, Boolean.class, Boolean.TRUE)).booleanValue();
   }
 
   @Override // ClientProtocol
   public void rename2(final String src, final String dst,
       final Options.Rename... options) throws IOException {
-    clientProto.rename2(src, dst, options);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations =
+        getLocationsForPath(src, true, false);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+          " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename2",
+        new Class<?>[] {String.class, String.class, options.getClass()},
+        new RemoteParam(), dstParam, options);
+    rpcClient.invokeSequential(locs, method, null, null);
   }
 
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
-    clientProto.concat(trg, src);
+    checkOperation(OperationCategory.WRITE);
+
+    // See if the src and target files are all in the same namespace
+    LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
+    if (targetBlocks == null) {
+      throw new IOException("Cannot locate blocks for target file - " + trg);
+    }
+    LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
+    String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
+    for (String source : src) {
+      LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
+      if (sourceBlocks == null) {
+        throw new IOException(
+            "Cannot located blocks for source file " + source);
+      }
+      String sourceBlockPoolId =
+          sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
+      if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
+        throw new IOException("Cannot concatenate source file " + source
+            + " because it is located in a different namespace"
+            + " with block pool id " + sourceBlockPoolId
+            + " from the target file with block pool id "
+            + targetBlockPoolId);
+      }
+    }
+
+    // Find locations in the matching namespace.
+    final RemoteLocation targetDestination =
+        getLocationForPath(trg, true, targetBlockPoolId);
+    String[] sourceDestinations = new String[src.length];
+    for (int i = 0; i < src.length; i++) {
+      String sourceFile = src[i];
+      RemoteLocation location =
+          getLocationForPath(sourceFile, true, targetBlockPoolId);
+      sourceDestinations[i] = location.getDest();
+    }
+    // Invoke
+    RemoteMethod method = new RemoteMethod("concat",
+        new Class<?>[] {String.class, String[].class},
+        targetDestination.getDest(), sourceDestinations);
+    rpcClient.invokeSingle(targetDestination, method);
   }
 
   @Override // ClientProtocol
   public boolean truncate(String src, long newLength, String clientName)
       throws IOException {
-    return clientProto.truncate(src, newLength, clientName);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("truncate",
+        new Class<?>[] {String.class, long.class, String.class},
+        new RemoteParam(), newLength, clientName);
+    return ((Boolean) rpcClient.invokeSequential(locations, method,
+        Boolean.class, Boolean.TRUE)).booleanValue();
   }
 
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
-    return clientProto.delete(src, recursive);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        getLocationsForPath(src, true, false);
+    RemoteMethod method = new RemoteMethod("delete",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        recursive);
+    if (isPathAll(src)) {
+      return rpcClient.invokeAll(locations, method);
+    } else {
+      return rpcClient.invokeSequential(locations, method,
+          Boolean.class, Boolean.TRUE).booleanValue();
+    }
   }
 
   @Override // ClientProtocol
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
-    return clientProto.mkdirs(src, masked, createParent);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("mkdirs",
+        new Class<?>[] {String.class, FsPermission.class, boolean.class},
+        new RemoteParam(), masked, createParent);
+
+    // Create in all locations
+    if (isPathAll(src)) {
+      return rpcClient.invokeAll(locations, method);
+    }
+
+    if (locations.size() > 1) {
+      // Check if this directory already exists
+      try {
+        HdfsFileStatus fileStatus = getFileInfo(src);
+        if (fileStatus != null) {
+          // When existing, the NN doesn't return an exception; return true
+          return true;
+        }
+      } catch (IOException ioe) {
+        // Can't query if this file exists or not.
+        LOG.error("Error requesting file info for path {} while proxing mkdirs",
+            src, ioe);
+      }
+    }
+
+    RemoteLocation firstLocation = locations.get(0);
+    return ((Boolean) rpcClient.invokeSingle(firstLocation, method))
+        .booleanValue();
   }
 
   @Override // ClientProtocol
   public void renewLease(String clientName) throws IOException {
-    clientProto.renewLease(clientName);
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("renewLease",
+        new Class<?>[] {String.class}, clientName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, false, false);
   }
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
       boolean needLocation) throws IOException {
-    return clientProto.getListing(src, startAfter, needLocation);
+    checkOperation(OperationCategory.READ);
+
+    // Locate the dir and fetch the listing
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getListing",
+        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
+        new RemoteParam(), startAfter, needLocation);
+    Map<RemoteLocation, DirectoryListing> listings =
+        rpcClient.invokeConcurrent(
+            locations, method, false, false, DirectoryListing.class);
+
+    Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
+    int totalRemainingEntries = 0;
+    int remainingEntries = 0;
+    boolean namenodeListingExists = false;
+    if (listings != null) {
+      // Check the subcluster listing with the smallest name
+      String lastName = null;
+      for (Entry<RemoteLocation, DirectoryListing> entry :
+          listings.entrySet()) {
+        RemoteLocation location = entry.getKey();
+        DirectoryListing listing = entry.getValue();
+        if (listing == null) {
+          LOG.debug("Cannot get listing from {}", location);
+        } else {
+          totalRemainingEntries += listing.getRemainingEntries();
+          HdfsFileStatus[] partialListing = listing.getPartialListing();
+          int length = partialListing.length;
+          if (length > 0) {
+            HdfsFileStatus lastLocalEntry = partialListing[length-1];
+            String lastLocalName = lastLocalEntry.getLocalName();
+            if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
+              lastName = lastLocalName;
+            }
+          }
+        }
+      }
+
+      // Add existing entries
+      for (Object value : listings.values()) {
+        DirectoryListing listing = (DirectoryListing) value;
+        if (listing != null) {
+          namenodeListingExists = true;
+          for (HdfsFileStatus file : listing.getPartialListing()) {
+            String filename = file.getLocalName();
+            if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
+              // Discarding entries further than the lastName
+              remainingEntries++;
+            } else {
+              nnListing.put(filename, file);
+            }
+          }
+          remainingEntries += listing.getRemainingEntries();
+        }
+      }
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(src);
+    if (children != null) {
+      // Get the dates for each mount point
+      Map<String, Long> dates = getMountPointDates(src);
+
+      // Create virtual folder with the mount name
+      for (String child : children) {
+        long date = 0;
+        if (dates != null && dates.containsKey(child)) {
+          date = dates.get(child);
+        }
+        // TODO add number of children
+        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
+
+        // This may overwrite existing listing entries with the mount point
+        // TODO don't add if already there?
+        nnListing.put(child, dirStatus);
+      }
+    }
+
+    if (!namenodeListingExists && nnListing.size() == 0) {
+      // NN returns a null object if the directory cannot be found and has no
+      // listing. If we didn't retrieve any NN listing data, and there are no
+      // mount points here, return null.
+      return null;
+    }
+
+    // Generate combined listing
+    HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
+    combinedData = nnListing.values().toArray(combinedData);
+    return new DirectoryListing(combinedData, remainingEntries);
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileInfo(String src) throws IOException {
-    return clientProto.getFileInfo(src);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+
+    HdfsFileStatus ret = null;
+    // If it's a directory, we check in all locations
+    if (isPathAll(src)) {
+      ret = getFileInfoAll(locations, method);
+    } else {
+      // Check for file information sequentially
+      ret = (HdfsFileStatus) rpcClient.invokeSequential(
+          locations, method, HdfsFileStatus.class, null);
+    }
+
+    // If there is no real path, check mount points
+    if (ret == null) {
+      List<String> children = subclusterResolver.getMountPoints(src);
+      if (children != null && !children.isEmpty()) {
+        Map<String, Long> dates = getMountPointDates(src);
+        long date = 0;
+        if (dates != null && dates.containsKey(src)) {
+          date = dates.get(src);
+        }
+        ret = getMountPointStatus(src, children.size(), date);
+      }
+    }
+
+    return ret;
+  }
+
+  /**
+   * Get the file info from all the locations.
+   *
+   * @param locations Locations to check.
+   * @param method The file information method to run.
+   * @return The first file info if it's a file, the directory if it's
+   *         everywhere.
+   * @throws IOException If all the locations throw an exception.
+   */
+  private HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
+      final RemoteMethod method) throws IOException {
+
+    // Get the file info from everybody
+    Map<RemoteLocation, HdfsFileStatus> results =
+        rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
+
+    // We return the first file
+    HdfsFileStatus dirStatus = null;
+    for (RemoteLocation loc : locations) {
+      HdfsFileStatus fileStatus = results.get(loc);
+      if (fileStatus != null) {
+        if (!fileStatus.isDirectory()) {
+          return fileStatus;
+        } else if (dirStatus == null) {
+          dirStatus = fileStatus;
+        }
+      }
+    }
+    return dirStatus;
   }
 
   @Override // ClientProtocol
   public boolean isFileClosed(String src) throws IOException {
-    return clientProto.isFileClosed(src);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("isFileClosed",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return ((Boolean) rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE)).booleanValue();
   }
 
   @Override // ClientProtocol
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
-    return clientProto.getFileLinkInfo(src);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileLinkInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (HdfsFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
   }
 
-  @Override // ClientProtocol
+  @Override
   public HdfsLocatedFileStatus getLocatedFileInfo(String src,
       boolean needBlockToken) throws IOException {
-    return clientProto.getLocatedFileInfo(src, needBlockToken);
+    checkOperation(OperationCategory.READ);
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        Boolean.valueOf(needBlockToken));
+    return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
   }
 
   @Override // ClientProtocol
   public long[] getStats() throws IOException {
-    return clientProto.getStats();
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("getStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, long[]> results =
+        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
+    long[] combinedData = new long[STATS_ARRAY_LENGTH];
+    for (long[] data : results.values()) {
+      for (int i = 0; i < combinedData.length && i < data.length; i++) {
+        if (data[i] >= 0) {
+          combinedData[i] += data[i];
+        }
+      }
+    }
+    return combinedData;
   }
 
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
       throws IOException {
-    return clientProto.getDatanodeReport(type);
+    checkOperation(OperationCategory.UNCHECKED);
+    return getDatanodeReport(type, true, 0);
   }
 
   /**
@@ -778,7 +1305,29 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public DatanodeStorageReport[] getDatanodeStorageReport(
       DatanodeReportType type) throws IOException {
-    return clientProto.getDatanodeStorageReport(type);
+    checkOperation(OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeStorageReport[]> dnSubcluster =
+        getDatanodeStorageReportMap(type);
+
+    // Avoid repeating machines in multiple subclusters
+    Map<String, DatanodeStorageReport> datanodesMap = new LinkedHashMap<>();
+    for (DatanodeStorageReport[] dns : dnSubcluster.values()) {
+      for (DatanodeStorageReport dn : dns) {
+        DatanodeInfo dnInfo = dn.getDatanodeInfo();
+        String nodeId = dnInfo.getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          datanodesMap.put(nodeId, dn);
+        }
+        // TODO merge somehow, right now it just takes the first one
+      }
+    }
+
+    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
+    DatanodeStorageReport[] combinedData =
+        new DatanodeStorageReport[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
   }
 
   /**
@@ -811,388 +1360,740 @@ public class RouterRpcServer extends AbstractService
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action, boolean isChecked)
       throws IOException {
-    return clientProto.setSafeMode(action, isChecked);
+    checkOperation(OperationCategory.WRITE);
+
+    // Set safe mode in all the name spaces
+    RemoteMethod method = new RemoteMethod("setSafeMode",
+        new Class<?>[] {SafeModeAction.class, boolean.class},
+        action, isChecked);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, !isChecked, Boolean.class);
+
+    // We only report true if all the name space are in safe mode
+    int numSafemode = 0;
+    for (boolean safemode : results.values()) {
+      if (safemode) {
+        numSafemode++;
+      }
+    }
+    return numSafemode == results.size();
   }
 
   @Override // ClientProtocol
   public boolean restoreFailedStorage(String arg) throws IOException {
-    return clientProto.restoreFailedStorage(arg);
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("restoreFailedStorage",
+        new Class<?>[] {String.class}, arg);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class);
+
+    boolean success = true;
+    for (boolean s : ret.values()) {
+      if (!s) {
+        success = false;
+        break;
+      }
+    }
+    return success;
   }
 
   @Override // ClientProtocol
   public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
-    return clientProto.saveNamespace(timeWindow, txGap);
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("saveNamespace",
+        new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
+
+    boolean success = true;
+    for (boolean s : ret.values()) {
+      if (!s) {
+        success = false;
+        break;
+      }
+    }
+    return success;
   }
 
   @Override // ClientProtocol
   public long rollEdits() throws IOException {
-    return clientProto.rollEdits();
+    checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+
+    // Return the maximum txid
+    long txid = 0;
+    for (long t : ret.values()) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
-    clientProto.refreshNodes();
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, true);
   }
 
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
-    clientProto.finalizeUpgrade();
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("finalizeUpgrade",
+        new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
   }
 
   @Override // ClientProtocol
   public boolean upgradeStatus() throws IOException {
-    return clientProto.upgradeStatus();
+    String methodName = getMethodName();
+    throw new UnsupportedOperationException(
+        "Operation \"" + methodName + "\" is not supported");
   }
 
   @Override // ClientProtocol
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
       throws IOException {
-    return clientProto.rollingUpgrade(action);
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("rollingUpgrade",
+        new Class<?>[] {RollingUpgradeAction.class}, action);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, RollingUpgradeInfo.class);
+
+    // Return the first rolling upgrade info
+    RollingUpgradeInfo info = null;
+    for (RollingUpgradeInfo infoNs : ret.values()) {
+      if (info == null && infoNs != null) {
+        info = infoNs;
+      }
+    }
+    return info;
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
-    clientProto.metaSave(filename);
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("metaSave",
+        new Class<?>[] {String.class}, filename);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
-    return clientProto.listCorruptFileBlocks(path, cookie);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(path, false);
+    RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), cookie);
+    return (CorruptFileBlocks) rpcClient.invokeSequential(
+        locations, method, CorruptFileBlocks.class, null);
   }
 
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    clientProto.setBalancerBandwidth(bandwidth);
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
+        new Class<?>[] {Long.class}, bandwidth);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
   }
 
   @Override // ClientProtocol
   public ContentSummary getContentSummary(String path) throws IOException {
-    return clientProto.getContentSummary(path);
+    checkOperation(OperationCategory.READ);
+
+    // Get the summaries from regular files
+    Collection<ContentSummary> summaries = new LinkedList<>();
+    FileNotFoundException notFoundException = null;
+    try {
+      final List<RemoteLocation> locations = getLocationsForPath(path, false);
+      RemoteMethod method = new RemoteMethod("getContentSummary",
+          new Class<?>[] {String.class}, new RemoteParam());
+      Map<RemoteLocation, ContentSummary> results =
+          rpcClient.invokeConcurrent(
+              locations, method, false, false, ContentSummary.class);
+      summaries.addAll(results.values());
+    } catch (FileNotFoundException e) {
+      notFoundException = e;
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(path);
+    if (children != null) {
+      for (String child : children) {
+        Path childPath = new Path(path, child);
+        try {
+          ContentSummary mountSummary = getContentSummary(childPath.toString());
+          if (mountSummary != null) {
+            summaries.add(mountSummary);
+          }
+        } catch (Exception e) {
+          LOG.error("Cannot get content summary for mount {}: {}",
+              childPath, e.getMessage());
+        }
+      }
+    }
+
+    // Throw original exception if no original nor mount points
+    if (summaries.isEmpty() && notFoundException != null) {
+      throw notFoundException;
+    }
+
+    return aggregateContentSummary(summaries);
+  }
+
+  /**
+   * Aggregate content summaries for each subcluster.
+   *
+   * @param summaries Collection of individual summaries.
+   * @return Aggregated content summary.
+   */
+  private ContentSummary aggregateContentSummary(
+      Collection<ContentSummary> summaries) {
+    if (summaries.size() == 1) {
+      return summaries.iterator().next();
+    }
+
+    long length = 0;
+    long fileCount = 0;
+    long directoryCount = 0;
+    long quota = 0;
+    long spaceConsumed = 0;
+    long spaceQuota = 0;
+
+    for (ContentSummary summary : summaries) {
+      length += summary.getLength();
+      fileCount += summary.getFileCount();
+      directoryCount += summary.getDirectoryCount();
+      quota += summary.getQuota();
+      spaceConsumed += summary.getSpaceConsumed();
+      spaceQuota += summary.getSpaceQuota();
+    }
+
+    ContentSummary ret = new ContentSummary.Builder()
+        .length(length)
+        .fileCount(fileCount)
+        .directoryCount(directoryCount)
+        .quota(quota)
+        .spaceConsumed(spaceConsumed)
+        .spaceQuota(spaceQuota)
+        .build();
+    return ret;
   }
 
   @Override // ClientProtocol
   public void fsync(String src, long fileId, String clientName,
       long lastBlockLength) throws IOException {
-    clientProto.fsync(src, fileId, clientName, lastBlockLength);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("fsync",
+        new Class<?>[] {String.class, long.class, String.class, long.class },
+        new RemoteParam(), fileId, clientName, lastBlockLength);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public void setTimes(String src, long mtime, long atime) throws IOException {
-    clientProto.setTimes(src, mtime, atime);
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setTimes",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), mtime, atime);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public void createSymlink(String target, String link, FsPermission dirPerms,
       boolean createParent) throws IOException {
-    clientProto.createSymlink(target, link, dirPerms, createParent);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO Verify that the link location is in the same NS as the targets
+    final List<RemoteLocation> targetLocations =
+        getLocationsForPath(target, true);
+    final List<RemoteLocation> linkLocations =
+        getLocationsForPath(link, true);
+    RemoteLocation linkLocation = linkLocations.get(0);
+    RemoteMethod method = new RemoteMethod("createSymlink",
+        new Class<?>[] {String.class, String.class, FsPermission.class,
+                        boolean.class},
+        new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
+    rpcClient.invokeSequential(targetLocations, method);
   }
 
   @Override // ClientProtocol
   public String getLinkTarget(String path) throws IOException {
-    return clientProto.getLinkTarget(path);
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("getLinkTarget",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (String) rpcClient.invokeSequential(
+        locations, method, String.class, null);
   }
 
   @Override // Client Protocol
   public void allowSnapshot(String snapshotRoot) throws IOException {
-    clientProto.allowSnapshot(snapshotRoot);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // Client Protocol
   public void disallowSnapshot(String snapshot) throws IOException {
-    clientProto.disallowSnapshot(snapshot);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException {
-    clientProto.renameSnapshot(snapshotRoot, snapshotOldName, snapshotNewName);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // Client Protocol
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
-    return clientProto.getSnapshottableDirListing();
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
       String earlierSnapshotName, String laterSnapshotName) throws IOException {
-    return clientProto.getSnapshotDiffReport(
-        snapshotRoot, earlierSnapshotName, laterSnapshotName);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public SnapshotDiffReportListing getSnapshotDiffReportListing(
       String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
       byte[] startPath, int index) throws IOException {
-    return clientProto.getSnapshotDiffReportListing(snapshotRoot,
-        earlierSnapshotName, laterSnapshotName, startPath, index);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public long addCacheDirective(CacheDirectiveInfo path,
       EnumSet<CacheFlag> flags) throws IOException {
-    return clientProto.addCacheDirective(path, flags);
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
   }
 
   @Override // ClientProtocol
   public void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags) throws IOException {
-    clientProto.modifyCacheDirective(directive, flags);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public void removeCacheDirective(long id) throws IOException {
-    clientProto.removeCacheDirective(id);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
       long prevId, CacheDirectiveInfo filter) throws IOException {
-    return clientProto.listCacheDirectives(prevId, filter);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public void addCachePool(CachePoolInfo info) throws IOException {
-    clientProto.addCachePool(info);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public void modifyCachePool(CachePoolInfo info) throws IOException {
-    clientProto.modifyCachePool(info);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public void removeCachePool(String cachePoolName) throws IOException {
-    clientProto.removeCachePool(cachePoolName);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
       throws IOException {
-    return clientProto.listCachePools(prevKey);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    clientProto.modifyAclEntries(src, aclSpec);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("modifyAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
   }
 
   @Override // ClienProtocol
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
-    clientProto.removeAclEntries(src, aclSpec);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
   }
 
   @Override // ClientProtocol
   public void removeDefaultAcl(String src) throws IOException {
-    clientProto.removeDefaultAcl(src);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeDefaultAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public void removeAcl(String src) throws IOException {
-    clientProto.removeAcl(src);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
-    clientProto.setAcl(src, aclSpec);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod(
+        "setAcl", new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public AclStatus getAclStatus(String src) throws IOException {
-    return clientProto.getAclStatus(src);
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAclStatus",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (AclStatus) rpcClient.invokeSequential(
+        locations, method, AclStatus.class, null);
   }
 
   @Override // ClientProtocol
   public void createEncryptionZone(String src, String keyName)
       throws IOException {
-    clientProto.createEncryptionZone(src, keyName);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("createEncryptionZone",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), keyName);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public EncryptionZone getEZForPath(String src) throws IOException {
-    return clientProto.getEZForPath(src);
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getEZForPath",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (EncryptionZone) rpcClient.invokeSequential(
+        locations, method, EncryptionZone.class, null);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
-    return clientProto.listEncryptionZones(prevId);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public void reencryptEncryptionZone(String zone, ReencryptAction action)
       throws IOException {
-    clientProto.reencryptEncryptionZone(zone, action);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(
       long prevId) throws IOException {
-    return clientProto.listReencryptionStatus(prevId);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
-    clientProto.setXAttr(src, xAttr, flag);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setXAttr",
+        new Class<?>[] {String.class, XAttr.class, EnumSet.class},
+        new RemoteParam(), xAttr, flag);
+    rpcClient.invokeSequential(locations, method);
   }
 
+  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
-    return clientProto.getXAttrs(src, xAttrs);
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getXAttrs",
+        new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
   }
 
+  @SuppressWarnings("unchecked")
   @Override // ClientProtocol
   public List<XAttr> listXAttrs(String src) throws IOException {
-    return clientProto.listXAttrs(src);
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("listXAttrs",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
   }
 
   @Override // ClientProtocol
   public void removeXAttr(String src, XAttr xAttr) throws IOException {
-    clientProto.removeXAttr(src, xAttr);
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeXAttr",
+        new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public void checkAccess(String path, FsAction mode) throws IOException {
-    clientProto.checkAccess(path, mode);
+    checkOperation(OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("checkAccess",
+        new Class<?>[] {String.class, FsAction.class},
+        new RemoteParam(), mode);
+    rpcClient.invokeSequential(locations, method);
   }
 
   @Override // ClientProtocol
   public long getCurrentEditLogTxid() throws IOException {
-    return clientProto.getCurrentEditLogTxid();
+    checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod(
+        "getCurrentEditLogTxid", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+
+    // Return the maximum txid
+    long txid = 0;
+    for (long t : ret.values()) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
   }
 
   @Override // ClientProtocol
   public EventBatchList getEditsFromTxid(long txid) throws IOException {
-    return clientProto.getEditsFromTxid(txid);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
-  @Override // ClientProtocol
+  @Override
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
-    return clientProto.getDataEncryptionKey();
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
-  @Override // ClientProtocol
+  @Override
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    return clientProto.createSnapshot(snapshotRoot, snapshotName);
+    checkOperation(OperationCategory.WRITE);
+    return null;
   }
 
-  @Override // ClientProtocol
+  @Override
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
-    clientProto.deleteSnapshot(snapshotRoot, snapshotName);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
-    clientProto.setQuota(path, namespaceQuota, storagespaceQuota, type);
+    this.quotaCall.setQuota(path, namespaceQuota, storagespaceQuota, type);
   }
 
   @Override // ClientProtocol
   public QuotaUsage getQuotaUsage(String path) throws IOException {
-    return clientProto.getQuotaUsage(path);
+    return this.quotaCall.getQuotaUsage(path);
   }
 
-  @Override // ClientProtocol
+  @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    clientProto.reportBadBlocks(blocks);
+    checkOperation(OperationCategory.WRITE);
+
+    // Block pool id -> blocks
+    Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
+    for (LocatedBlock block : blocks) {
+      String bpId = block.getBlock().getBlockPoolId();
+      List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
+      if (bpBlocks == null) {
+        bpBlocks = new LinkedList<>();
+        blockLocations.put(bpId, bpBlocks);
+      }
+      bpBlocks.add(block);
+    }
+
+    // Invoke each block pool
+    for (Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
+      String bpId = entry.getKey();
+      List<LocatedBlock> bpBlocks = entry.getValue();
+
+      LocatedBlock[] bpBlocksArray =
+          bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
+      RemoteMethod method = new RemoteMethod("reportBadBlocks",
+          new Class<?>[] {LocatedBlock[].class},
+          new Object[] {bpBlocksArray});
+      rpcClient.invokeSingleBlockPool(bpId, method);
+    }
   }
 
-  @Override // ClientProtocol
+  @Override
   public void unsetStoragePolicy(String src) throws IOException {
-    clientProto.unsetStoragePolicy(src);
+    checkOperation(OperationCategory.WRITE, false);
   }
 
-  @Override // ClientProtocol
+  @Override
   public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
-    return clientProto.getStoragePolicy(path);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
       throws IOException {
-    return clientProto.getErasureCodingPolicies();
+    return erasureCoding.getErasureCodingPolicies();
   }
 
   @Override // ClientProtocol
   public Map<String, String> getErasureCodingCodecs() throws IOException {
-    return clientProto.getErasureCodingCodecs();
+    return erasureCoding.getErasureCodingCodecs();
   }
 
   @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    return clientProto.addErasureCodingPolicies(policies);
+    return erasureCoding.addErasureCodingPolicies(policies);
   }
 
   @Override // ClientProtocol
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    clientProto.removeErasureCodingPolicy(ecPolicyName);
+    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void disableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    clientProto.disableErasureCodingPolicy(ecPolicyName);
+    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void enableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
-    clientProto.enableErasureCodingPolicy(ecPolicyName);
+    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
   }
 
   @Override // ClientProtocol
   public ErasureCodingPolicy getErasureCodingPolicy(String src)
       throws IOException {
-    return clientProto.getErasureCodingPolicy(src);
+    return erasureCoding.getErasureCodingPolicy(src);
   }
 
   @Override // ClientProtocol
   public void setErasureCodingPolicy(String src, String ecPolicyName)
       throws IOException {
-    clientProto.setErasureCodingPolicy(src, ecPolicyName);
+    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
   }
 
   @Override // ClientProtocol
   public void unsetErasureCodingPolicy(String src) throws IOException {
-    clientProto.unsetErasureCodingPolicy(src);
+    erasureCoding.unsetErasureCodingPolicy(src);
   }
 
-  @Override // ClientProtocol
+  @Override
   public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    return clientProto.getECBlockGroupStats();
+    return erasureCoding.getECBlockGroupStats();
   }
 
-  @Override // ClientProtocol
+  @Override
   public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
-    return clientProto.getReplicatedBlockStats();
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Deprecated
-  @Override // ClientProtocol
+  @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return clientProto.listOpenFiles(prevId);
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
-  @Override // ClientProtocol
+  @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
       EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
-    return clientProto.listOpenFiles(prevId, openFilesTypes, path);
-  }
-
-  @Override // ClientProtocol
-  public void satisfyStoragePolicy(String path) throws IOException {
-    clientProto.satisfyStoragePolicy(path);
+    checkOperation(OperationCategory.READ, false);
+    return null;
   }
 
   @Override // NamenodeProtocol
@@ -1266,11 +2167,6 @@ public class RouterRpcServer extends AbstractService
     return nnProto.isRollingUpgrade();
   }
 
-  @Override // NamenodeProtocol
-  public Long getNextSPSPath() throws IOException {
-    return nnProto.getNextSPSPath();
-  }
-
   /**
    * Locate the location with the matching block pool id.
    *
@@ -1280,7 +2176,7 @@ public class RouterRpcServer extends AbstractService
    * @return Prioritized list of locations in the federated cluster.
    * @throws IOException if the location for this path cannot be determined.
    */
-  protected RemoteLocation getLocationForPath(
+  private RemoteLocation getLocationForPath(
       String path, boolean failIfLocked, String blockPoolId)
           throws IOException {
 
@@ -1380,6 +2276,27 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
+   * Check if a path should be in all subclusters.
+   *
+   * @param path Path to check.
+   * @return If a path should be in all subclusters.
+   */
+  private boolean isPathAll(final String path) {
+    if (subclusterResolver instanceof MountTableResolver) {
+      try {
+        MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+        MountTable entry = mountTable.getMountPoint(path);
+        if (entry != null) {
+          return entry.isAll();
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get mount point", e);
+      }
+    }
+    return false;
+  }
+
+  /**
    * Check if a path is in a read only mount point.
    *
    * @param path Path to check.
@@ -1401,6 +2318,121 @@ public class RouterRpcServer extends AbstractService
   }
 
   /**
+   * Get the modification dates for mount points.
+   *
+   * @param path Name of the path to start checking dates from.
+   * @return Map with the modification dates for all sub-entries.
+   */
+  private Map<String, Long> getMountPointDates(String path) {
+    Map<String, Long> ret = new TreeMap<>();
+    if (subclusterResolver instanceof MountTableResolver) {
+      try {
+        final List<String> children = subclusterResolver.getMountPoints(path);
+        for (String child : children) {
+          Long modTime = getModifiedTime(ret, path, child);
+          ret.put(child, modTime);
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get mount point", e);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Get modified time for child. If the child is present in mount table it
+   * will return the modified time. If the child is not present but subdirs of
+   * this child are present then it will return latest modified subdir's time
+   * as modified time of the requested child.
+   * @param ret contains children and modified times.
+   * @param mountTable.
+   * @param path Name of the path to start checking dates from.
+   * @param child child of the requested path.
+   * @return modified time.
+   */
+  private long getModifiedTime(Map<String, Long> ret, String path,
+      String child) {
+    MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+    String srcPath;
+    if (path.equals(Path.SEPARATOR)) {
+      srcPath = Path.SEPARATOR + child;
+    } else {
+      srcPath = path + Path.SEPARATOR + child;
+    }
+    Long modTime = 0L;
+    try {
+      // Get mount table entry for the srcPath
+      MountTable entry = mountTable.getMountPoint(srcPath);
+      // if srcPath is not in mount table but its subdirs are in mount
+      // table we will display latest modified subdir date/time.
+      if (entry == null) {
+        List<MountTable> entries = mountTable.getMounts(srcPath);
+        for (MountTable eachEntry : entries) {
+          // Get the latest date
+          if (ret.get(child) == null ||
+              ret.get(child) < eachEntry.getDateModified()) {
+            modTime = eachEntry.getDateModified();
+          }
+        }
+      } else {
+        modTime = entry.getDateModified();
+      }
+    } catch (IOException e) {
+      LOG.error("Cannot get mount point", e);
+    }
+    return modTime;
+  }
+
+  /**
+   * Create a new file status for a mount point.
+   *
+   * @param name Name of the mount point.
+   * @param childrenNum Number of children.
+   * @param date Map with the dates.
+   * @return New HDFS file status representing a mount point.
+   */
+  private HdfsFileStatus getMountPointStatus(
+      String name, int childrenNum, long date) {
+    long modTime = date;
+    long accessTime = date;
+    FsPermission permission = FsPermission.getDirDefault();
+    String owner = this.superUser;
+    String group = this.superGroup;
+    try {
+      // TODO support users, it should be the user for the pointed folder
+      UserGroupInformation ugi = getRemoteUser();
+      owner = ugi.getUserName();
+      group = ugi.getPrimaryGroupName();
+    } catch (IOException e) {
+      LOG.error("Cannot get the remote user: {}", e.getMessage());
+    }
+    long inodeId = 0;
+    return new HdfsFileStatus.Builder()
+      .isdir(true)
+      .mtime(modTime)
+      .atime(accessTime)
+      .perm(permission)
+      .owner(owner)
+      .group(group)
+      .symlink(new byte[0])
+      .path(DFSUtil.string2Bytes(name))
+      .fileId(inodeId)
+      .children(childrenNum)
+      .build();
+  }
+
+  /**
+   * Get the name of the method that is calling this function.
+   *
+   * @return Name of the method calling this function.
+   */
+  private static String getMethodName() {
+    final StackTraceElement[] stack = Thread.currentThread().getStackTrace();
+    String methodName = stack[3].getMethodName();
+    return methodName;
+  }
+
+  /**
    * Get the user that is invoking this operation.
    *
    * @return Remote user group information.
@@ -1458,4 +2490,16 @@ public class RouterRpcServer extends AbstractService
   public FederationRPCMetrics getRPCMetrics() {
     return this.rpcMonitor.getRPCMetrics();
   }
+
+  @Override
+  public void satisfyStoragePolicy(String path) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public Long getNextSPSPath() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    // not supported
+    return null;
+  }
 }


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


[31/50] [abbrv] hadoop git commit: HDDS-265. Move numPendingDeletionBlocks and deleteTransactionId from ContainerData to KeyValueContainerData. Contributed by LiXin Ge.

Posted by sh...@apache.org.
HDDS-265. Move numPendingDeletionBlocks and deleteTransactionId from ContainerData to KeyValueContainerData. Contributed by LiXin Ge.


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

Branch: refs/heads/HDFS-12943
Commit: 5aa15cfaffbf294b5025989c20d905b01da52c2b
Parents: 8184739
Author: Hanisha Koneru <ha...@apache.org>
Authored: Wed Aug 22 10:04:15 2018 -0700
Committer: Hanisha Koneru <ha...@apache.org>
Committed: Wed Aug 22 10:04:15 2018 -0700

----------------------------------------------------------------------
 .../common/helpers/ContainerReport.java         |  14 ---
 .../common/helpers/KeyValueContainerReport.java | 117 +++++++++++++++++++
 .../container/common/impl/ContainerData.java    |  53 ---------
 .../container/common/impl/ContainerSet.java     |  50 +-------
 .../RandomContainerDeletionChoosingPolicy.java  |   3 +-
 ...NOrderedContainerDeletionChoosingPolicy.java |  24 ++--
 .../container/common/interfaces/Container.java  |   6 +
 .../ContainerDeletionChoosingPolicy.java        |  13 +++
 .../container/keyvalue/KeyValueContainer.java   |  46 ++++++++
 .../keyvalue/KeyValueContainerData.java         |  54 +++++++++
 .../common/TestKeyValueContainerData.java       |   4 +-
 .../common/TestBlockDeletingService.java        |   8 +-
 .../commandhandler/TestBlockDeletion.java       |   9 +-
 13 files changed, 268 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
index b242754..a4c1f2f 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.helpers;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
 
-import static java.lang.Math.max;
 
 /**
  * Container Report iterates the closed containers and sends a container report
@@ -37,7 +36,6 @@ public class ContainerReport {
   private long readBytes;
   private long writeBytes;
   private long containerID;
-  private long deleteTransactionId;
 
   public long getContainerID() {
     return containerID;
@@ -47,9 +45,6 @@ public class ContainerReport {
     this.containerID = containerID;
   }
 
-
-
-
   /**
    * Constructs the ContainerReport.
    *
@@ -66,7 +61,6 @@ public class ContainerReport {
     this.readBytes = 0L;
     this.writeCount = 0L;
     this.writeBytes = 0L;
-    this.deleteTransactionId = 0;
   }
 
   /**
@@ -100,9 +94,6 @@ public class ContainerReport {
     if (info.hasWriteBytes()) {
       report.setWriteBytes(info.getWriteBytes());
     }
-    if (info.hasDeleteTransactionId()) {
-      report.updateDeleteTransactionId(info.getDeleteTransactionId());
-    }
 
     report.setContainerID(info.getContainerID());
     return report;
@@ -193,10 +184,6 @@ public class ContainerReport {
     this.bytesUsed = bytesUsed;
   }
 
-  public void updateDeleteTransactionId(long transactionId) {
-    this.deleteTransactionId = max(transactionId, deleteTransactionId);
-  }
-
   /**
    * Gets a containerInfo protobuf message from ContainerReports.
    *
@@ -213,7 +200,6 @@ public class ContainerReport {
         .setWriteBytes(this.getWriteBytes())
         .setFinalhash(this.getFinalhash())
         .setContainerID(this.getContainerID())
-        .setDeleteTransactionId(this.deleteTransactionId)
         .build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
new file mode 100644
index 0000000..b03487b
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
@@ -0,0 +1,117 @@
+/*
+ * 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.container.common.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
+
+import static java.lang.Math.max;
+
+/**
+ * KeyValueContainer Report iterates the closed containers and sends a
+ * container report to SCM.
+ */
+public class KeyValueContainerReport extends ContainerReport{
+  private long deleteTransactionId;
+
+  /**
+   * Constructs the KeyValueContainerReport.
+   *
+   * @param containerID - Container ID.
+   * @param finalhash - Final Hash.
+   */
+  public KeyValueContainerReport(long containerID, String finalhash) {
+    super(containerID, finalhash);
+    this.deleteTransactionId = 0;
+  }
+
+  /**
+   * Sets the deleteTransactionId if it is greater than existing.
+   * @param transactionId - deleteTransactionId
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    this.deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Gets the deleteTransactionId.
+   * @return - deleteTransactionId.
+   */
+  public long getDeleteTransactionId() {
+    return this.deleteTransactionId;
+  }
+
+  /**
+   * Gets a containerReport from protobuf class.
+   *
+   * @param info - ContainerInfo.
+   * @return - ContainerReport.
+   */
+  public static KeyValueContainerReport getFromProtoBuf(ContainerInfo info) {
+    Preconditions.checkNotNull(info);
+    KeyValueContainerReport report = new KeyValueContainerReport(
+        info.getContainerID(), info.getFinalhash());
+    if (info.hasSize()) {
+      report.setSize(info.getSize());
+    }
+    if (info.hasKeyCount()) {
+      report.setKeyCount(info.getKeyCount());
+    }
+    if (info.hasUsed()) {
+      report.setBytesUsed(info.getUsed());
+    }
+    if (info.hasReadCount()) {
+      report.setReadCount(info.getReadCount());
+    }
+    if (info.hasReadBytes()) {
+      report.setReadBytes(info.getReadBytes());
+    }
+    if (info.hasWriteCount()) {
+      report.setWriteCount(info.getWriteCount());
+    }
+    if (info.hasWriteBytes()) {
+      report.setWriteBytes(info.getWriteBytes());
+    }
+    if (info.hasDeleteTransactionId()) {
+      report.updateDeleteTransactionId(info.getDeleteTransactionId());
+    }
+    report.setContainerID(info.getContainerID());
+    return report;
+  }
+
+  /**
+   * Gets a containerInfo protobuf message from ContainerReports.
+   *
+   * @return ContainerInfo
+   */
+  @Override
+  public ContainerInfo getProtoBufMessage() {
+    return ContainerInfo.newBuilder()
+        .setKeyCount(this.getKeyCount())
+        .setSize(this.getSize())
+        .setUsed(this.getBytesUsed())
+        .setReadCount(this.getReadCount())
+        .setReadBytes(this.getReadBytes())
+        .setWriteCount(this.getWriteCount())
+        .setWriteBytes(this.getWriteBytes())
+        .setFinalhash(this.getFinalhash())
+        .setContainerID(this.getContainerID())
+        .setDeleteTransactionId(this.getDeleteTransactionId())
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
index 26954a7..47894dc 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java
@@ -33,11 +33,9 @@ import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
 import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import org.yaml.snakeyaml.Yaml;
 
-import static java.lang.Math.max;
 import static org.apache.hadoop.ozone.OzoneConsts.CHECKSUM;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ID;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_TYPE;
@@ -81,8 +79,6 @@ public abstract class ContainerData {
 
   private HddsVolume volume;
 
-  private long deleteTransactionId;
-
   private String checksum;
   public static final Charset CHARSET_ENCODING = Charset.forName("UTF-8");
   private static final String DUMMY_CHECKSUM = new String(new byte[64],
@@ -99,12 +95,6 @@ public abstract class ContainerData {
       MAX_SIZE_GB,
       CHECKSUM));
 
-
-  /**
-   * Number of pending deletion blocks in container.
-   */
-  private final AtomicInteger numPendingDeletionBlocks;
-
   /**
    * Creates a ContainerData Object, which holds metadata of the container.
    * @param type - ContainerType
@@ -139,8 +129,6 @@ public abstract class ContainerData {
     this.bytesUsed = new AtomicLong(0L);
     this.keyCount = new AtomicLong(0L);
     this.maxSizeGB = size;
-    this.numPendingDeletionBlocks = new AtomicInteger(0);
-    this.deleteTransactionId = 0;
     setChecksumTo0ByteArray();
   }
 
@@ -403,31 +391,6 @@ public abstract class ContainerData {
     this.keyCount.set(count);
   }
 
-  /**
-   * Increase the count of pending deletion blocks.
-   *
-   * @param numBlocks increment number
-   */
-  public void incrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks.addAndGet(numBlocks);
-  }
-
-  /**
-   * Decrease the count of pending deletion blocks.
-   *
-   * @param numBlocks decrement number
-   */
-  public void decrPendingDeletionBlocks(int numBlocks) {
-    this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
-  }
-
-  /**
-   * Get the number of pending deletion blocks.
-   */
-  public int getNumPendingDeletionBlocks() {
-    return this.numPendingDeletionBlocks.get();
-  }
-
   public void setChecksumTo0ByteArray() {
     this.checksum = DUMMY_CHECKSUM;
   }
@@ -469,20 +432,4 @@ public abstract class ContainerData {
    * @return Protocol Buffer Message
    */
   public abstract ContainerProtos.ContainerData getProtoBufMessage();
-
-  /**
-   * Sets deleteTransactionId to latest delete transactionId for the container.
-   *
-   * @param transactionId latest transactionId of the container.
-   */
-  public void updateDeleteTransactionId(long transactionId) {
-    deleteTransactionId = max(transactionId, deleteTransactionId);
-  }
-
-  /**
-   * Return the latest deleteTransactionId of the container.
-   */
-  public long getDeleteTransactionId() {
-    return deleteTransactionId;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index 3da09f2..f92ab52 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -22,9 +22,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
 import org.apache.hadoop.hdds.scm.container.common.helpers
@@ -43,8 +40,6 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.stream.Collectors;
 
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Result.INVALID_CONTAINER_STATE;
 
 /**
  * Class that manages Containers created on the datanode.
@@ -204,58 +199,19 @@ public class ContainerSet {
     ContainerReportsProto.Builder crBuilder =
         ContainerReportsProto.newBuilder();
 
-
     for (Container container: containers) {
-      long containerId = container.getContainerData().getContainerID();
-      ContainerInfo.Builder ciBuilder = ContainerInfo.newBuilder();
-      ContainerData containerData = container.getContainerData();
-      ciBuilder.setContainerID(containerId)
-          .setReadCount(containerData.getReadCount())
-          .setWriteCount(containerData.getWriteCount())
-          .setReadBytes(containerData.getReadBytes())
-          .setWriteBytes(containerData.getWriteBytes())
-          .setUsed(containerData.getBytesUsed())
-          .setState(getState(containerData))
-          .setDeleteTransactionId(containerData.getDeleteTransactionId());
-
-      crBuilder.addReports(ciBuilder.build());
+      crBuilder.addReports(container.getContainerReport());
     }
 
     return crBuilder.build();
   }
 
-  /**
-   * Returns LifeCycle State of the container.
-   * @param containerData - ContainerData
-   * @return LifeCycle State of the container
-   * @throws StorageContainerException
-   */
-  private HddsProtos.LifeCycleState getState(ContainerData containerData)
-      throws StorageContainerException {
-    HddsProtos.LifeCycleState state;
-    switch (containerData.getState()) {
-    case OPEN:
-      state = HddsProtos.LifeCycleState.OPEN;
-      break;
-    case CLOSING:
-      state = HddsProtos.LifeCycleState.CLOSING;
-      break;
-    case CLOSED:
-      state = HddsProtos.LifeCycleState.CLOSED;
-      break;
-    default:
-      throw new StorageContainerException("Invalid Container state found: " +
-          containerData.getContainerID(), INVALID_CONTAINER_STATE);
-    }
-    return state;
-  }
-
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
       ContainerDeletionChoosingPolicy deletionPolicy)
       throws StorageContainerException {
     Map<Long, ContainerData> containerDataMap = containerMap.entrySet().stream()
-        .filter(e -> e.getValue().getContainerType()
-            == ContainerProtos.ContainerType.KeyValueContainer)
+        .filter(e -> deletionPolicy.isValidContainerType(
+            e.getValue().getContainerType()))
         .collect(Collectors.toMap(Map.Entry::getKey,
             e -> e.getValue().getContainerData()));
     return deletionPolicy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
index 83d746b..5c6c319 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/RandomContainerDeletionChoosingPolicy.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,7 +59,7 @@ public class RandomContainerDeletionChoosingPolicy
         LOG.debug("Select container {} for block deletion, "
             + "pending deletion blocks num: {}.",
             entry.getContainerID(),
-            entry.getNumPendingDeletionBlocks());
+            ((KeyValueContainerData)entry).getNumPendingDeletionBlocks());
       } else {
         break;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
index 68074fc..b17680c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.interfaces
     .ContainerDeletionChoosingPolicy;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,14 +42,11 @@ public class TopNOrderedContainerDeletionChoosingPolicy
       LoggerFactory.getLogger(TopNOrderedContainerDeletionChoosingPolicy.class);
 
   /** customized comparator used to compare differentiate container data. **/
-  private static final Comparator<ContainerData> CONTAINER_DATA_COMPARATOR
-      = new Comparator<ContainerData>() {
-        @Override
-        public int compare(ContainerData c1, ContainerData c2) {
-          return Integer.compare(c2.getNumPendingDeletionBlocks(),
-              c1.getNumPendingDeletionBlocks());
-        }
-      };
+  private static final Comparator<KeyValueContainerData>
+        KEY_VALUE_CONTAINER_DATA_COMPARATOR = (KeyValueContainerData c1,
+                                               KeyValueContainerData c2) ->
+              Integer.compare(c2.getNumPendingDeletionBlocks(),
+                  c1.getNumPendingDeletionBlocks());
 
   @Override
   public List<ContainerData> chooseContainerForBlockDeletion(int count,
@@ -58,13 +56,15 @@ public class TopNOrderedContainerDeletionChoosingPolicy
         "Internal assertion: candidate containers cannot be null");
 
     List<ContainerData> result = new LinkedList<>();
-    List<ContainerData> orderedList = new LinkedList<>();
-    orderedList.addAll(candidateContainers.values());
-    Collections.sort(orderedList, CONTAINER_DATA_COMPARATOR);
+    List<KeyValueContainerData> orderedList = new LinkedList<>();
+    for (ContainerData entry : candidateContainers.values()) {
+      orderedList.add((KeyValueContainerData)entry);
+    }
+    Collections.sort(orderedList, KEY_VALUE_CONTAINER_DATA_COMPARATOR);
 
     // get top N list ordered by pending deletion blocks' number
     int currentCount = 0;
-    for (ContainerData entry : orderedList) {
+    for (KeyValueContainerData entry : orderedList) {
       if (currentCount < count) {
         if (entry.getNumPendingDeletionBlocks() > 0) {
           result.add(entry);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index a7077d9..7f706b5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.container.common.interfaces;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.
     StorageContainerException;
 
@@ -111,4 +112,9 @@ public interface Container extends RwLock {
    */
   BlockIterator blockIterator() throws IOException;
 
+  /**
+   * Returns containerReport for the container.
+   */
+  StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()
+      throws StorageContainerException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
index dce86e9..84c4f90 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/ContainerDeletionChoosingPolicy.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.container.common.interfaces;
 
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -42,4 +43,16 @@ public interface ContainerDeletionChoosingPolicy {
   List<ContainerData> chooseContainerForBlockDeletion(int count,
       Map<Long, ContainerData> candidateContainers)
       throws StorageContainerException;
+
+  /**
+   * Determine if the container has suitable type for this policy.
+   * @param type  type of the container
+   * @return whether the container type suitable for this policy.
+   */
+  default boolean isValidContainerType(ContainerProtos.ContainerType type) {
+    if (type == ContainerProtos.ContainerType.KeyValueContainer) {
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index c96f997..0ea748a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerLifeCycleState;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers
     .StorageContainerException;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -406,6 +408,50 @@ public class KeyValueContainer implements Container {
   }
 
   /**
+   * Returns KeyValueContainerReport for the KeyValueContainer.
+   */
+  @Override
+  public StorageContainerDatanodeProtocolProtos.ContainerInfo
+      getContainerReport() throws StorageContainerException{
+    StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
+        StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
+    ciBuilder.setContainerID(containerData.getContainerID())
+        .setReadCount(containerData.getReadCount())
+        .setWriteCount(containerData.getWriteCount())
+        .setReadBytes(containerData.getReadBytes())
+        .setWriteBytes(containerData.getWriteBytes())
+        .setUsed(containerData.getBytesUsed())
+        .setState(getHddsState())
+        .setDeleteTransactionId(containerData.getDeleteTransactionId());
+    return ciBuilder.build();
+  }
+
+  /**
+   * Returns LifeCycle State of the container.
+   * @return LifeCycle State of the container in HddsProtos format
+   * @throws StorageContainerException
+   */
+  private HddsProtos.LifeCycleState getHddsState()
+      throws StorageContainerException {
+    HddsProtos.LifeCycleState state;
+    switch (containerData.getState()) {
+    case OPEN:
+      state = HddsProtos.LifeCycleState.OPEN;
+      break;
+    case CLOSING:
+      state = HddsProtos.LifeCycleState.CLOSING;
+      break;
+    case CLOSED:
+      state = HddsProtos.LifeCycleState.CLOSED;
+      break;
+    default:
+      throw new StorageContainerException("Invalid Container state found: " +
+          containerData.getContainerID(), INVALID_CONTAINER_STATE);
+    }
+    return state;
+  }
+
+  /**
    * Returns container DB file.
    * @return
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 0705cf4..1d37437 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -32,7 +32,9 @@ import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import static java.lang.Math.max;
 import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE;
 import static org.apache.hadoop.ozone.OzoneConsts.METADATA_PATH;
@@ -61,6 +63,13 @@ public class KeyValueContainerData extends ContainerData {
 
   private File dbFile = null;
 
+  /**
+   * Number of pending deletion blocks in KeyValueContainer.
+   */
+  private final AtomicInteger numPendingDeletionBlocks;
+
+  private long deleteTransactionId;
+
   static {
     // Initialize YAML fields
     KV_YAML_FIELDS = Lists.newArrayList();
@@ -77,6 +86,8 @@ public class KeyValueContainerData extends ContainerData {
    */
   public KeyValueContainerData(long id, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, size);
+    this.numPendingDeletionBlocks = new AtomicInteger(0);
+    this.deleteTransactionId = 0;
   }
 
   /**
@@ -88,6 +99,8 @@ public class KeyValueContainerData extends ContainerData {
   public KeyValueContainerData(long id, int layOutVersion, int size) {
     super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
         size);
+    this.numPendingDeletionBlocks = new AtomicInteger(0);
+    this.deleteTransactionId = 0;
   }
 
 
@@ -169,6 +182,47 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   /**
+   * Increase the count of pending deletion blocks.
+   *
+   * @param numBlocks increment number
+   */
+  public void incrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(numBlocks);
+  }
+
+  /**
+   * Decrease the count of pending deletion blocks.
+   *
+   * @param numBlocks decrement number
+   */
+  public void decrPendingDeletionBlocks(int numBlocks) {
+    this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
+  }
+
+  /**
+   * Get the number of pending deletion blocks.
+   */
+  public int getNumPendingDeletionBlocks() {
+    return this.numPendingDeletionBlocks.get();
+  }
+
+  /**
+   * Sets deleteTransactionId to latest delete transactionId for the container.
+   *
+   * @param transactionId latest transactionId of the container.
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Return the latest deleteTransactionId of the container.
+   */
+  public long getDeleteTransactionId() {
+    return deleteTransactionId;
+  }
+
+  /**
    * Returns a ProtoBuf Message from ContainerData.
    *
    * @return Protocol Buffer Message

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
index 42db66d..12ce163 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestKeyValueContainerData.java
@@ -57,6 +57,7 @@ public class TestKeyValueContainerData {
     assertEquals(val.get(), kvData.getReadCount());
     assertEquals(val.get(), kvData.getWriteCount());
     assertEquals(val.get(), kvData.getKeyCount());
+    assertEquals(val.get(), kvData.getNumPendingDeletionBlocks());
     assertEquals(MAXSIZE, kvData.getMaxSizeGB());
 
     kvData.setState(state);
@@ -68,6 +69,7 @@ public class TestKeyValueContainerData {
     kvData.incrReadCount();
     kvData.incrWriteCount();
     kvData.incrKeyCount();
+    kvData.incrPendingDeletionBlocks(1);
 
     assertEquals(state, kvData.getState());
     assertEquals(containerDBType, kvData.getContainerDBType());
@@ -79,7 +81,7 @@ public class TestKeyValueContainerData {
     assertEquals(1, kvData.getReadCount());
     assertEquals(1, kvData.getWriteCount());
     assertEquals(1, kvData.getKeyCount());
-
+    assertEquals(1, kvData.getNumPendingDeletionBlocks());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
index 4ca4124..dcf4022 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/TestBlockDeletingService.java
@@ -203,8 +203,12 @@ public class TestBlockDeletingService {
     MetadataStore meta = KeyUtils.getDB(
         (KeyValueContainerData) containerData.get(0), conf);
     Map<Long, Container> containerMap = containerSet.getContainerMap();
-    long transactionId = containerMap.get(containerData.get(0).getContainerID())
-        .getContainerData().getDeleteTransactionId();
+    // NOTE: this test assumes that all the container is KetValueContainer and
+    // have DeleteTransactionId in KetValueContainerData. If other
+    // types is going to be added, this test should be checked.
+    long transactionId = ((KeyValueContainerData)containerMap
+        .get(containerData.get(0).getContainerID()).getContainerData())
+        .getDeleteTransactionId();
 
 
     // Number of deleted blocks in container should be equal to 0 before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5aa15cfa/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 2524de6..94cdf61 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -139,7 +139,9 @@ public class TestBlockDeletion {
     Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList));
     // No containers with deleted blocks
     Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
-    // Delete transactionIds for the containers should be 0
+    // Delete transactionIds for the containers should be 0.
+    // NOTE: this test assumes that all the container is KetValueContainer. If
+    // other container types is going to be added, this test should be checked.
     matchContainerTransactionIds();
     om.deleteKey(keyArgs);
     Thread.sleep(5000);
@@ -215,8 +217,9 @@ public class TestBlockDeletion {
         Assert.assertEquals(
             scm.getContainerInfo(containerId).getDeleteTransactionId(), 0);
       }
-      Assert.assertEquals(dnContainerSet.getContainer(containerId)
-              .getContainerData().getDeleteTransactionId(),
+      Assert.assertEquals(((KeyValueContainerData)dnContainerSet
+              .getContainer(containerId).getContainerData())
+              .getDeleteTransactionId(),
           scm.getContainerInfo(containerId).getDeleteTransactionId());
     }
   }


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


[16/50] [abbrv] hadoop git commit: HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun.

Posted by sh...@apache.org.
HDFS-13790. RBF: Move ClientProtocol APIs to its own module. Contributed by Chao Sun.


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

Branch: refs/heads/HDFS-12943
Commit: 6425ed27ea638da75f656204d6df4adad1d91fe1
Parents: 4aacbff
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Mon Aug 20 12:20:29 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Mon Aug 20 12:20:29 2018 +0530

----------------------------------------------------------------------
 .../federation/router/RouterClientProtocol.java | 1805 ++++++++++++++++++
 .../federation/router/RouterRpcServer.java      | 1360 ++-----------
 2 files changed, 1963 insertions(+), 1202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6425ed27/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
new file mode 100644
index 0000000..f45da3c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
@@ -0,0 +1,1805 @@
+/**
+ * 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.hdfs.server.federation.router;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
+import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * Module that implements all the RPC calls in {@link ClientProtocol} in the
+ * {@link RouterRpcServer}.
+ */
+public class RouterClientProtocol implements ClientProtocol {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterClientProtocol.class.getName());
+
+  private final RouterRpcServer rpcServer;
+  private final RouterRpcClient rpcClient;
+  private final FileSubclusterResolver subclusterResolver;
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** Identifier for the super user. */
+  private final String superUser;
+  /** Identifier for the super group. */
+  private final String superGroup;
+  /** Erasure coding calls. */
+  private final ErasureCoding erasureCoding;
+
+  RouterClientProtocol(Configuration conf, RouterRpcServer rpcServer) {
+    this.rpcServer = rpcServer;
+    this.rpcClient = rpcServer.getRPCClient();
+    this.subclusterResolver = rpcServer.getSubclusterResolver();
+    this.namenodeResolver = rpcServer.getNamenodeResolver();
+
+    // User and group for reporting
+    this.superUser = System.getProperty("user.name");
+    this.superGroup = conf.get(
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY,
+        DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
+    this.erasureCoding = new ErasureCoding(rpcServer);
+  }
+
+  @Override
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+    return null;
+  }
+
+  /**
+   * The the delegation token from each name service.
+   *
+   * @param renewer
+   * @return Name service -> Token.
+   * @throws IOException
+   */
+  public Map<FederationNamespaceInfo, Token<DelegationTokenIdentifier>>
+  getDelegationTokens(Text renewer) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+    return null;
+  }
+
+  @Override
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public LocatedBlocks getBlockLocations(String src, final long offset,
+      final long length) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, false);
+    RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), offset, length);
+    return rpcClient.invokeSequential(locations, remoteMethod,
+        LocatedBlocks.class, null);
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getServerDefaults");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (FsServerDefaults) rpcClient.invokeSingle(ns, method);
+  }
+
+  @Override
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize,
+      CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    if (createParent && isPathAll(src)) {
+      int index = src.lastIndexOf(Path.SEPARATOR);
+      String parent = src.substring(0, index);
+      LOG.debug("Creating {} requires creating parent {}", src, parent);
+      FsPermission parentPermissions = getParentPermission(masked);
+      boolean success = mkdirs(parent, parentPermissions, createParent);
+      if (!success) {
+        // This shouldn't happen as mkdirs returns true or exception
+        LOG.error("Couldn't create parents for {}", src);
+      }
+    }
+
+    RemoteLocation createLocation = rpcServer.getCreateLocation(src);
+    RemoteMethod method = new RemoteMethod("create",
+        new Class<?>[] {String.class, FsPermission.class, String.class,
+            EnumSetWritable.class, boolean.class, short.class,
+            long.class, CryptoProtocolVersion[].class,
+            String.class},
+        createLocation.getDest(), masked, clientName, flag, createParent,
+        replication, blockSize, supportedVersions, ecPolicyName);
+    return (HdfsFileStatus) rpcClient.invokeSingle(createLocation, method);
+  }
+
+  @Override
+  public LastBlockWithStatus append(String src, final String clientName,
+      final EnumSetWritable<CreateFlag> flag) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("append",
+        new Class<?>[] {String.class, String.class, EnumSetWritable.class},
+        new RemoteParam(), clientName, flag);
+    return rpcClient.invokeSequential(
+        locations, method, LastBlockWithStatus.class, null);
+  }
+
+  @Override
+  public boolean recoverLease(String src, String clientName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("recoverLease",
+        new Class<?>[] {String.class, String.class}, new RemoteParam(),
+        clientName);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
+  }
+
+  @Override
+  public boolean setReplication(String src, short replication)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setReplication",
+        new Class<?>[] {String.class, short.class}, new RemoteParam(),
+        replication);
+    Object result = rpcClient.invokeSequential(
+        locations, method, Boolean.class, Boolean.TRUE);
+    return (boolean) result;
+  }
+
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    List<RemoteLocation> locations = rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setStoragePolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), policyName);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getStoragePolicies");
+    String ns = subclusterResolver.getDefaultNamespace();
+    return (BlockStoragePolicy[]) rpcClient.invokeSingle(ns, method);
+  }
+
+  @Override
+  public void setPermission(String src, FsPermission permissions)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setPermission",
+        new Class<?>[] {String.class, FsPermission.class},
+        new RemoteParam(), permissions);
+    if (isPathAll(src)) {
+      rpcClient.invokeConcurrent(locations, method);
+    } else {
+      rpcClient.invokeSequential(locations, method);
+    }
+  }
+
+  @Override
+  public void setOwner(String src, String username, String groupname)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setOwner",
+        new Class<?>[] {String.class, String.class, String.class},
+        new RemoteParam(), username, groupname);
+    if (isPathAll(src)) {
+      rpcClient.invokeConcurrent(locations, method);
+    } else {
+      rpcClient.invokeSequential(locations, method);
+    }
+  }
+
+  /**
+   * Excluded and favored nodes are not verified and will be ignored by
+   * placement policy if they are not in the same nameservice as the file.
+   */
+  @Override
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludedNodes, long fileId,
+      String[] favoredNodes, EnumSet<AddBlockFlag> addBlockFlags)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("addBlock",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+            DatanodeInfo[].class, long.class, String[].class,
+            EnumSet.class},
+        new RemoteParam(), clientName, previous, excludedNodes, fileId,
+        favoredNodes, addBlockFlags);
+    // TODO verify the excludedNodes and favoredNodes are acceptable to this NN
+    return rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
+  }
+
+  /**
+   * Excluded nodes are not verified and will be ignored by placement if they
+   * are not in the same nameservice as the file.
+   */
+  @Override
+  public LocatedBlock getAdditionalDatanode(final String src, final long fileId,
+      final ExtendedBlock blk, final DatanodeInfo[] existings,
+      final String[] existingStorageIDs, final DatanodeInfo[] excludes,
+      final int numAdditionalNodes, final String clientName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAdditionalDatanode",
+        new Class<?>[] {String.class, long.class, ExtendedBlock.class,
+            DatanodeInfo[].class, String[].class,
+            DatanodeInfo[].class, int.class, String.class},
+        new RemoteParam(), fileId, blk, existings, existingStorageIDs, excludes,
+        numAdditionalNodes, clientName);
+    return rpcClient.invokeSequential(
+        locations, method, LocatedBlock.class, null);
+  }
+
+  @Override
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+      String holder) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("abandonBlock",
+        new Class<?>[] {ExtendedBlock.class, long.class, String.class,
+            String.class},
+        b, fileId, new RemoteParam(), holder);
+    rpcClient.invokeSingle(b, method);
+  }
+
+  @Override
+  public boolean complete(String src, String clientName, ExtendedBlock last,
+      long fileId) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("complete",
+        new Class<?>[] {String.class, String.class, ExtendedBlock.class,
+            long.class},
+        new RemoteParam(), clientName, last, fileId);
+    // Complete can return true/false, so don't expect a result
+    return rpcClient.invokeSequential(locations, method, Boolean.class, null);
+  }
+
+  @Override
+  public LocatedBlock updateBlockForPipeline(
+      ExtendedBlock block, String clientName) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updateBlockForPipeline",
+        new Class<?>[] {ExtendedBlock.class, String.class},
+        block, clientName);
+    return (LocatedBlock) rpcClient.invokeSingle(block, method);
+  }
+
+  /**
+   * Datanode are not verified to be in the same nameservice as the old block.
+   * TODO This may require validation.
+   */
+  @Override
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("updatePipeline",
+        new Class<?>[] {String.class, ExtendedBlock.class, ExtendedBlock.class,
+            DatanodeID[].class, String[].class},
+        clientName, oldBlock, newBlock, newNodes, newStorageIDs);
+    rpcClient.invokeSingle(oldBlock, method);
+  }
+
+  @Override
+  public long getPreferredBlockSize(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getPreferredBlockSize",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(locations, method, Long.class, null);
+  }
+
+  @Deprecated
+  @Override
+  public boolean rename(final String src, final String dst)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations =
+        rpcServer.getLocationsForPath(src, true, false);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+              " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), dstParam);
+    return rpcClient.invokeSequential(locs, method, Boolean.class,
+        Boolean.TRUE);
+  }
+
+  @Override
+  public void rename2(final String src, final String dst,
+      final Options.Rename... options) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations =
+        rpcServer.getLocationsForPath(src, true, false);
+    // srcLocations may be trimmed by getRenameDestinations()
+    final List<RemoteLocation> locs = new LinkedList<>(srcLocations);
+    RemoteParam dstParam = getRenameDestinations(locs, dst);
+    if (locs.isEmpty()) {
+      throw new IOException(
+          "Rename of " + src + " to " + dst + " is not allowed," +
+              " no eligible destination in the same namespace was found.");
+    }
+    RemoteMethod method = new RemoteMethod("rename2",
+        new Class<?>[] {String.class, String.class, options.getClass()},
+        new RemoteParam(), dstParam, options);
+    rpcClient.invokeSequential(locs, method, null, null);
+  }
+
+  @Override
+  public void concat(String trg, String[] src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // See if the src and target files are all in the same namespace
+    LocatedBlocks targetBlocks = getBlockLocations(trg, 0, 1);
+    if (targetBlocks == null) {
+      throw new IOException("Cannot locate blocks for target file - " + trg);
+    }
+    LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock();
+    String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId();
+    for (String source : src) {
+      LocatedBlocks sourceBlocks = getBlockLocations(source, 0, 1);
+      if (sourceBlocks == null) {
+        throw new IOException(
+            "Cannot located blocks for source file " + source);
+      }
+      String sourceBlockPoolId =
+          sourceBlocks.getLastLocatedBlock().getBlock().getBlockPoolId();
+      if (!sourceBlockPoolId.equals(targetBlockPoolId)) {
+        throw new IOException("Cannot concatenate source file " + source
+            + " because it is located in a different namespace"
+            + " with block pool id " + sourceBlockPoolId
+            + " from the target file with block pool id "
+            + targetBlockPoolId);
+      }
+    }
+
+    // Find locations in the matching namespace.
+    final RemoteLocation targetDestination =
+        rpcServer.getLocationForPath(trg, true, targetBlockPoolId);
+    String[] sourceDestinations = new String[src.length];
+    for (int i = 0; i < src.length; i++) {
+      String sourceFile = src[i];
+      RemoteLocation location =
+          rpcServer.getLocationForPath(sourceFile, true, targetBlockPoolId);
+      sourceDestinations[i] = location.getDest();
+    }
+    // Invoke
+    RemoteMethod method = new RemoteMethod("concat",
+        new Class<?>[] {String.class, String[].class},
+        targetDestination.getDest(), sourceDestinations);
+    rpcClient.invokeSingle(targetDestination, method);
+  }
+
+  @Override
+  public boolean truncate(String src, long newLength, String clientName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("truncate",
+        new Class<?>[] {String.class, long.class, String.class},
+        new RemoteParam(), newLength, clientName);
+    return rpcClient.invokeSequential(locations, method, Boolean.class,
+        Boolean.TRUE);
+  }
+
+  @Override
+  public boolean delete(String src, boolean recursive) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true, false);
+    RemoteMethod method = new RemoteMethod("delete",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        recursive);
+    if (isPathAll(src)) {
+      return rpcClient.invokeAll(locations, method);
+    } else {
+      return rpcClient.invokeSequential(locations, method,
+          Boolean.class, Boolean.TRUE);
+    }
+  }
+
+  @Override
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("mkdirs",
+        new Class<?>[] {String.class, FsPermission.class, boolean.class},
+        new RemoteParam(), masked, createParent);
+
+    // Create in all locations
+    if (isPathAll(src)) {
+      return rpcClient.invokeAll(locations, method);
+    }
+
+    if (locations.size() > 1) {
+      // Check if this directory already exists
+      try {
+        HdfsFileStatus fileStatus = getFileInfo(src);
+        if (fileStatus != null) {
+          // When existing, the NN doesn't return an exception; return true
+          return true;
+        }
+      } catch (IOException ioe) {
+        // Can't query if this file exists or not.
+        LOG.error("Error requesting file info for path {} while proxing mkdirs",
+            src, ioe);
+      }
+    }
+
+    RemoteLocation firstLocation = locations.get(0);
+    return (boolean) rpcClient.invokeSingle(firstLocation, method);
+  }
+
+  @Override
+  public void renewLease(String clientName) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("renewLease",
+        new Class<?>[] {String.class}, clientName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, false, false);
+  }
+
+  @Override
+  public DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // Locate the dir and fetch the listing
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("getListing",
+        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
+        new RemoteParam(), startAfter, needLocation);
+    Map<RemoteLocation, DirectoryListing> listings =
+        rpcClient.invokeConcurrent(
+            locations, method, false, false, DirectoryListing.class);
+
+    Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
+    int totalRemainingEntries = 0;
+    int remainingEntries = 0;
+    boolean namenodeListingExists = false;
+    if (listings != null) {
+      // Check the subcluster listing with the smallest name
+      String lastName = null;
+      for (Map.Entry<RemoteLocation, DirectoryListing> entry :
+          listings.entrySet()) {
+        RemoteLocation location = entry.getKey();
+        DirectoryListing listing = entry.getValue();
+        if (listing == null) {
+          LOG.debug("Cannot get listing from {}", location);
+        } else {
+          totalRemainingEntries += listing.getRemainingEntries();
+          HdfsFileStatus[] partialListing = listing.getPartialListing();
+          int length = partialListing.length;
+          if (length > 0) {
+            HdfsFileStatus lastLocalEntry = partialListing[length-1];
+            String lastLocalName = lastLocalEntry.getLocalName();
+            if (lastName == null || lastName.compareTo(lastLocalName) > 0) {
+              lastName = lastLocalName;
+            }
+          }
+        }
+      }
+
+      // Add existing entries
+      for (Object value : listings.values()) {
+        DirectoryListing listing = (DirectoryListing) value;
+        if (listing != null) {
+          namenodeListingExists = true;
+          for (HdfsFileStatus file : listing.getPartialListing()) {
+            String filename = file.getLocalName();
+            if (totalRemainingEntries > 0 && filename.compareTo(lastName) > 0) {
+              // Discarding entries further than the lastName
+              remainingEntries++;
+            } else {
+              nnListing.put(filename, file);
+            }
+          }
+          remainingEntries += listing.getRemainingEntries();
+        }
+      }
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(src);
+    if (children != null) {
+      // Get the dates for each mount point
+      Map<String, Long> dates = getMountPointDates(src);
+
+      // Create virtual folder with the mount name
+      for (String child : children) {
+        long date = 0;
+        if (dates != null && dates.containsKey(child)) {
+          date = dates.get(child);
+        }
+        // TODO add number of children
+        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
+
+        // This may overwrite existing listing entries with the mount point
+        // TODO don't add if already there?
+        nnListing.put(child, dirStatus);
+      }
+    }
+
+    if (!namenodeListingExists && nnListing.size() == 0) {
+      // NN returns a null object if the directory cannot be found and has no
+      // listing. If we didn't retrieve any NN listing data, and there are no
+      // mount points here, return null.
+      return null;
+    }
+
+    // Generate combined listing
+    HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()];
+    combinedData = nnListing.values().toArray(combinedData);
+    return new DirectoryListing(combinedData, remainingEntries);
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+
+    HdfsFileStatus ret = null;
+    // If it's a directory, we check in all locations
+    if (isPathAll(src)) {
+      ret = getFileInfoAll(locations, method);
+    } else {
+      // Check for file information sequentially
+      ret = rpcClient.invokeSequential(
+          locations, method, HdfsFileStatus.class, null);
+    }
+
+    // If there is no real path, check mount points
+    if (ret == null) {
+      List<String> children = subclusterResolver.getMountPoints(src);
+      if (children != null && !children.isEmpty()) {
+        Map<String, Long> dates = getMountPointDates(src);
+        long date = 0;
+        if (dates != null && dates.containsKey(src)) {
+          date = dates.get(src);
+        }
+        ret = getMountPointStatus(src, children.size(), date);
+      }
+    }
+
+    return ret;
+  }
+
+  @Override
+  public boolean isFileClosed(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("isFileClosed",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(locations, method, Boolean.class,
+        Boolean.TRUE);
+  }
+
+  @Override
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileLinkInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(locations, method, HdfsFileStatus.class,
+        null);
+  }
+
+  @Override
+  public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+      boolean needBlockToken) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        needBlockToken);
+    return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
+  }
+
+  @Override
+  public long[] getStats() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("getStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, long[]> results =
+        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
+    long[] combinedData = new long[STATS_ARRAY_LENGTH];
+    for (long[] data : results.values()) {
+      for (int i = 0; i < combinedData.length && i < data.length; i++) {
+        if (data[i] >= 0) {
+          combinedData[i] += data[i];
+        }
+      }
+    }
+    return combinedData;
+  }
+
+  @Override
+  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+    return rpcServer.getDatanodeReport(type, true, 0);
+  }
+
+  @Override
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      HdfsConstants.DatanodeReportType type) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeStorageReport[]> dnSubcluster =
+        rpcServer.getDatanodeStorageReportMap(type);
+
+    // Avoid repeating machines in multiple subclusters
+    Map<String, DatanodeStorageReport> datanodesMap = new LinkedHashMap<>();
+    for (DatanodeStorageReport[] dns : dnSubcluster.values()) {
+      for (DatanodeStorageReport dn : dns) {
+        DatanodeInfo dnInfo = dn.getDatanodeInfo();
+        String nodeId = dnInfo.getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          datanodesMap.put(nodeId, dn);
+        }
+        // TODO merge somehow, right now it just takes the first one
+      }
+    }
+
+    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
+    DatanodeStorageReport[] combinedData =
+        new DatanodeStorageReport[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
+  }
+
+  @Override
+  public boolean setSafeMode(HdfsConstants.SafeModeAction action,
+      boolean isChecked) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // Set safe mode in all the name spaces
+    RemoteMethod method = new RemoteMethod("setSafeMode",
+        new Class<?>[] {HdfsConstants.SafeModeAction.class, boolean.class},
+        action, isChecked);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, !isChecked, Boolean.class);
+
+    // We only report true if all the name space are in safe mode
+    int numSafemode = 0;
+    for (boolean safemode : results.values()) {
+      if (safemode) {
+        numSafemode++;
+      }
+    }
+    return numSafemode == results.size();
+  }
+
+  @Override
+  public boolean restoreFailedStorage(String arg) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("restoreFailedStorage",
+        new Class<?>[] {String.class}, arg);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class);
+
+    boolean success = true;
+    for (boolean s : ret.values()) {
+      if (!s) {
+        success = false;
+        break;
+      }
+    }
+    return success;
+  }
+
+  @Override
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("saveNamespace",
+        new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
+
+    boolean success = true;
+    for (boolean s : ret.values()) {
+      if (!s) {
+        success = false;
+        break;
+      }
+    }
+    return success;
+  }
+
+  @Override
+  public long rollEdits() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+
+    // Return the maximum txid
+    long txid = 0;
+    for (long t : ret.values()) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @Override
+  public void refreshNodes() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("refreshNodes", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, true);
+  }
+
+  @Override
+  public void finalizeUpgrade() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("finalizeUpgrade",
+        new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override
+  public boolean upgradeStatus() throws IOException {
+    String methodName = RouterRpcServer.getMethodName();
+    throw new UnsupportedOperationException(
+        "Operation \"" + methodName + "\" is not supported");
+  }
+
+  @Override
+  public RollingUpgradeInfo rollingUpgrade(HdfsConstants.RollingUpgradeAction action)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("rollingUpgrade",
+        new Class<?>[] {HdfsConstants.RollingUpgradeAction.class}, action);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, RollingUpgradeInfo.class);
+
+    // Return the first rolling upgrade info
+    RollingUpgradeInfo info = null;
+    for (RollingUpgradeInfo infoNs : ret.values()) {
+      if (info == null && infoNs != null) {
+        info = infoNs;
+      }
+    }
+    return info;
+  }
+
+  @Override
+  public void metaSave(String filename) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("metaSave",
+        new Class<?>[] {String.class}, filename);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(path, false);
+    RemoteMethod method = new RemoteMethod("listCorruptFileBlocks",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), cookie);
+    return rpcClient.invokeSequential(
+        locations, method, CorruptFileBlocks.class, null);
+  }
+
+  @Override
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("setBalancerBandwidth",
+        new Class<?>[] {Long.class}, bandwidth);
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  @Override
+  public ContentSummary getContentSummary(String path) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // Get the summaries from regular files
+    Collection<ContentSummary> summaries = new LinkedList<>();
+    FileNotFoundException notFoundException = null;
+    try {
+      final List<RemoteLocation> locations =
+          rpcServer.getLocationsForPath(path, false);
+      RemoteMethod method = new RemoteMethod("getContentSummary",
+          new Class<?>[] {String.class}, new RemoteParam());
+      Map<RemoteLocation, ContentSummary> results =
+          rpcClient.invokeConcurrent(
+              locations, method, false, false, ContentSummary.class);
+      summaries.addAll(results.values());
+    } catch (FileNotFoundException e) {
+      notFoundException = e;
+    }
+
+    // Add mount points at this level in the tree
+    final List<String> children = subclusterResolver.getMountPoints(path);
+    if (children != null) {
+      for (String child : children) {
+        Path childPath = new Path(path, child);
+        try {
+          ContentSummary mountSummary = getContentSummary(childPath.toString());
+          if (mountSummary != null) {
+            summaries.add(mountSummary);
+          }
+        } catch (Exception e) {
+          LOG.error("Cannot get content summary for mount {}: {}",
+              childPath, e.getMessage());
+        }
+      }
+    }
+
+    // Throw original exception if no original nor mount points
+    if (summaries.isEmpty() && notFoundException != null) {
+      throw notFoundException;
+    }
+
+    return aggregateContentSummary(summaries);
+  }
+
+  @Override
+  public void fsync(String src, long fileId, String clientName,
+      long lastBlockLength) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("fsync",
+        new Class<?>[] {String.class, long.class, String.class, long.class },
+        new RemoteParam(), fileId, clientName, lastBlockLength);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public void setTimes(String src, long mtime, long atime) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setTimes",
+        new Class<?>[] {String.class, long.class, long.class},
+        new RemoteParam(), mtime, atime);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public void createSymlink(String target, String link, FsPermission dirPerms,
+      boolean createParent) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO Verify that the link location is in the same NS as the targets
+    final List<RemoteLocation> targetLocations =
+        rpcServer.getLocationsForPath(target, true);
+    final List<RemoteLocation> linkLocations =
+        rpcServer.getLocationsForPath(link, true);
+    RemoteLocation linkLocation = linkLocations.get(0);
+    RemoteMethod method = new RemoteMethod("createSymlink",
+        new Class<?>[] {String.class, String.class, FsPermission.class,
+            boolean.class},
+        new RemoteParam(), linkLocation.getDest(), dirPerms, createParent);
+    rpcClient.invokeSequential(targetLocations, method);
+  }
+
+  @Override
+  public String getLinkTarget(String path) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("getLinkTarget",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(locations, method, String.class, null);
+  }
+
+  @Override // Client Protocol
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override // Client Protocol
+  public void disallowSnapshot(String snapshot) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String earlierSnapshotName, String laterSnapshotName) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public SnapshotDiffReportListing getSnapshotDiffReportListing(
+      String snapshotRoot, String earlierSnapshotName, String laterSnapshotName,
+      byte[] startPath, int index) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public long addCacheDirective(CacheDirectiveInfo path,
+      EnumSet<CacheFlag> flags) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void removeCacheDirective(long id) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void removeCachePool(String cachePoolName) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("modifyAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAclEntries",
+        new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method, null, null);
+  }
+
+  @Override
+  public void removeDefaultAcl(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeDefaultAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public void removeAcl(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeAcl",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod(
+        "setAcl", new Class<?>[] {String.class, List.class},
+        new RemoteParam(), aclSpec);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public AclStatus getAclStatus(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getAclStatus",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(locations, method, AclStatus.class, null);
+  }
+
+  @Override
+  public void createEncryptionZone(String src, String keyName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("createEncryptionZone",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), keyName);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public EncryptionZone getEZForPath(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getEZForPath",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return rpcClient.invokeSequential(
+        locations, method, EncryptionZone.class, null);
+  }
+
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void reencryptEncryptionZone(String zone, HdfsConstants.ReencryptAction action)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(
+      long prevId) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setXAttr",
+        new Class<?>[] {String.class, XAttr.class, EnumSet.class},
+        new RemoteParam(), xAttr, flag);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getXAttrs",
+        new Class<?>[] {String.class, List.class}, new RemoteParam(), xAttrs);
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public List<XAttr> listXAttrs(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("listXAttrs",
+        new Class<?>[] {String.class}, new RemoteParam());
+    return (List<XAttr>) rpcClient.invokeSequential(
+        locations, method, List.class, null);
+  }
+
+  @Override
+  public void removeXAttr(String src, XAttr xAttr) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("removeXAttr",
+        new Class<?>[] {String.class, XAttr.class}, new RemoteParam(), xAttr);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public void checkAccess(String path, FsAction mode) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    // TODO handle virtual directories
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("checkAccess",
+        new Class<?>[] {String.class, FsAction.class},
+        new RemoteParam(), mode);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @Override
+  public long getCurrentEditLogTxid() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod(
+        "getCurrentEditLogTxid", new Class<?>[] {});
+    final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
+
+    // Return the maximum txid
+    long txid = 0;
+    for (long t : ret.values()) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @Override
+  public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+    return null;
+  }
+
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws IOException {
+    rpcServer.getQuotaModule()
+        .setQuota(path, namespaceQuota, storagespaceQuota, type);
+  }
+
+  @Override
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    return rpcServer.getQuotaModule().getQuotaUsage(path);
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE);
+
+    // Block pool id -> blocks
+    Map<String, List<LocatedBlock>> blockLocations = new HashMap<>();
+    for (LocatedBlock block : blocks) {
+      String bpId = block.getBlock().getBlockPoolId();
+      List<LocatedBlock> bpBlocks = blockLocations.get(bpId);
+      if (bpBlocks == null) {
+        bpBlocks = new LinkedList<>();
+        blockLocations.put(bpId, bpBlocks);
+      }
+      bpBlocks.add(block);
+    }
+
+    // Invoke each block pool
+    for (Map.Entry<String, List<LocatedBlock>> entry : blockLocations.entrySet()) {
+      String bpId = entry.getKey();
+      List<LocatedBlock> bpBlocks = entry.getValue();
+
+      LocatedBlock[] bpBlocksArray =
+          bpBlocks.toArray(new LocatedBlock[bpBlocks.size()]);
+      RemoteMethod method = new RemoteMethod("reportBadBlocks",
+          new Class<?>[] {LocatedBlock[].class},
+          new Object[] {bpBlocksArray});
+      rpcClient.invokeSingleBlockPool(bpId, method);
+    }
+  }
+
+  @Override
+  public void unsetStoragePolicy(String src) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicies();
+  }
+
+  @Override
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    return erasureCoding.getErasureCodingCodecs();
+  }
+
+  @Override
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    return erasureCoding.addErasureCodingPolicies(policies);
+  }
+
+  @Override
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
+  }
+
+  @Override
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
+  }
+
+  @Override
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
+  }
+
+  @Override
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicy(src);
+  }
+
+  @Override
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+  }
+
+  @Override
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    erasureCoding.unsetErasureCodingPolicy(src);
+  }
+
+  @Override
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    return erasureCoding.getECBlockGroupStats();
+  }
+
+  @Override
+  public ReplicatedBlockStats getReplicatedBlockStats() throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Deprecated
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
+      throws IOException {
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesIterator.OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
+  }
+
+  @Override
+  public BatchedRemoteIterator.BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesIterator.OpenFilesType> openFilesTypes, String path) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public void satisfyStoragePolicy(String path) throws IOException {
+    rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
+  }
+
+  /**
+   * Determines combinations of eligible src/dst locations for a rename. A
+   * rename cannot change the namespace. Renames are only allowed if there is an
+   * eligible dst location in the same namespace as the source.
+   *
+   * @param srcLocations List of all potential source destinations where the
+   *          path may be located. On return this list is trimmed to include
+   *          only the paths that have corresponding destinations in the same
+   *          namespace.
+   * @param dst The destination path
+   * @return A map of all eligible source namespaces and their corresponding
+   *         replacement value.
+   * @throws IOException If the dst paths could not be determined.
+   */
+  private RemoteParam getRenameDestinations(
+      final List<RemoteLocation> srcLocations, final String dst)
+      throws IOException {
+
+    final List<RemoteLocation> dstLocations =
+        rpcServer.getLocationsForPath(dst, true);
+    final Map<RemoteLocation, String> dstMap = new HashMap<>();
+
+    Iterator<RemoteLocation> iterator = srcLocations.iterator();
+    while (iterator.hasNext()) {
+      RemoteLocation srcLocation = iterator.next();
+      RemoteLocation eligibleDst =
+          getFirstMatchingLocation(srcLocation, dstLocations);
+      if (eligibleDst != null) {
+        // Use this dst for this source location
+        dstMap.put(srcLocation, eligibleDst.getDest());
+      } else {
+        // This src destination is not valid, remove from the source list
+        iterator.remove();
+      }
+    }
+    return new RemoteParam(dstMap);
+  }
+
+  /**
+   * Get first matching location.
+   *
+   * @param location Location we are looking for.
+   * @param locations List of locations.
+   * @return The first matchin location in the list.
+   */
+  private RemoteLocation getFirstMatchingLocation(RemoteLocation location,
+      List<RemoteLocation> locations) {
+    for (RemoteLocation loc : locations) {
+      if (loc.getNameserviceId().equals(location.getNameserviceId())) {
+        // Return first matching location
+        return loc;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Aggregate content summaries for each subcluster.
+   *
+   * @param summaries Collection of individual summaries.
+   * @return Aggregated content summary.
+   */
+  private ContentSummary aggregateContentSummary(
+      Collection<ContentSummary> summaries) {
+    if (summaries.size() == 1) {
+      return summaries.iterator().next();
+    }
+
+    long length = 0;
+    long fileCount = 0;
+    long directoryCount = 0;
+    long quota = 0;
+    long spaceConsumed = 0;
+    long spaceQuota = 0;
+
+    for (ContentSummary summary : summaries) {
+      length += summary.getLength();
+      fileCount += summary.getFileCount();
+      directoryCount += summary.getDirectoryCount();
+      quota += summary.getQuota();
+      spaceConsumed += summary.getSpaceConsumed();
+      spaceQuota += summary.getSpaceQuota();
+    }
+
+    ContentSummary ret = new ContentSummary.Builder()
+        .length(length)
+        .fileCount(fileCount)
+        .directoryCount(directoryCount)
+        .quota(quota)
+        .spaceConsumed(spaceConsumed)
+        .spaceQuota(spaceQuota)
+        .build();
+    return ret;
+  }
+
+  /**
+   * Get the file info from all the locations.
+   *
+   * @param locations Locations to check.
+   * @param method The file information method to run.
+   * @return The first file info if it's a file, the directory if it's
+   *         everywhere.
+   * @throws IOException If all the locations throw an exception.
+   */
+  private HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
+      final RemoteMethod method) throws IOException {
+
+    // Get the file info from everybody
+    Map<RemoteLocation, HdfsFileStatus> results =
+        rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
+
+    // We return the first file
+    HdfsFileStatus dirStatus = null;
+    for (RemoteLocation loc : locations) {
+      HdfsFileStatus fileStatus = results.get(loc);
+      if (fileStatus != null) {
+        if (!fileStatus.isDirectory()) {
+          return fileStatus;
+        } else if (dirStatus == null) {
+          dirStatus = fileStatus;
+        }
+      }
+    }
+    return dirStatus;
+  }
+
+  /**
+   * Get the permissions for the parent of a child with given permissions.
+   * Add implicit u+wx permission for parent. This is based on
+   * @{FSDirMkdirOp#addImplicitUwx}.
+   * @param mask The permission mask of the child.
+   * @return The permission mask of the parent.
+   */
+  private static FsPermission getParentPermission(final FsPermission mask) {
+    FsPermission ret = new FsPermission(
+        mask.getUserAction().or(FsAction.WRITE_EXECUTE),
+        mask.getGroupAction(),
+        mask.getOtherAction());
+    return ret;
+  }
+
+  /**
+   * Check if a path should be in all subclusters.
+   *
+   * @param path Path to check.
+   * @return If a path should be in all subclusters.
+   */
+  private boolean isPathAll(final String path) {
+    if (subclusterResolver instanceof MountTableResolver) {
+      try {
+        MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+        MountTable entry = mountTable.getMountPoint(path);
+        if (entry != null) {
+          return entry.isAll();
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get mount point", e);
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Create a new file status for a mount point.
+   *
+   * @param name Name of the mount point.
+   * @param childrenNum Number of children.
+   * @param date Map with the dates.
+   * @return New HDFS file status representing a mount point.
+   */
+  private HdfsFileStatus getMountPointStatus(
+      String name, int childrenNum, long date) {
+    long modTime = date;
+    long accessTime = date;
+    FsPermission permission = FsPermission.getDirDefault();
+    String owner = this.superUser;
+    String group = this.superGroup;
+    try {
+      // TODO support users, it should be the user for the pointed folder
+      UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+      owner = ugi.getUserName();
+      group = ugi.getPrimaryGroupName();
+    } catch (IOException e) {
+      LOG.error("Cannot get the remote user: {}", e.getMessage());
+    }
+    long inodeId = 0;
+    return new HdfsFileStatus.Builder()
+        .isdir(true)
+        .mtime(modTime)
+        .atime(accessTime)
+        .perm(permission)
+        .owner(owner)
+        .group(group)
+        .symlink(new byte[0])
+        .path(DFSUtil.string2Bytes(name))
+        .fileId(inodeId)
+        .children(childrenNum)
+        .build();
+  }
+
+  /**
+   * Get the modification dates for mount points.
+   *
+   * @param path Name of the path to start checking dates from.
+   * @return Map with the modification dates for all sub-entries.
+   */
+  private Map<String, Long> getMountPointDates(String path) {
+    Map<String, Long> ret = new TreeMap<>();
+    if (subclusterResolver instanceof MountTableResolver) {
+      try {
+        final List<String> children = subclusterResolver.getMountPoints(path);
+        for (String child : children) {
+          Long modTime = getModifiedTime(ret, path, child);
+          ret.put(child, modTime);
+        }
+      } catch (IOException e) {
+        LOG.error("Cannot get mount point", e);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Get modified time for child. If the child is present in mount table it
+   * will return the modified time. If the child is not present but subdirs of
+   * this child are present then it will return latest modified subdir's time
+   * as modified time of the requested child.
+   *
+   * @param ret contains children and modified times.
+   * @param path Name of the path to start checking dates from.
+   * @param child child of the requested path.
+   * @return modified time.
+   */
+  private long getModifiedTime(Map<String, Long> ret, String path,
+      String child) {
+    MountTableResolver mountTable = (MountTableResolver)subclusterResolver;
+    String srcPath;
+    if (path.equals(Path.SEPARATOR)) {
+      srcPath = Path.SEPARATOR + child;
+    } else {
+      srcPath = path + Path.SEPARATOR + child;
+    }
+    Long modTime = 0L;
+    try {
+      // Get mount table entry for the srcPath
+      MountTable entry = mountTable.getMountPoint(srcPath);
+      // if srcPath is not in mount table but its subdirs are in mount
+      // table we will display latest modified subdir date/time.
+      if (entry == null) {
+        List<MountTable> entries = mountTable.getMounts(srcPath);
+        for (MountTable eachEntry : entries) {
+          // Get the latest date
+          if (ret.get(child) == null ||
+              ret.get(child) < eachEntry.getDateModified()) {
+            modTime = eachEntry.getDateModified();
+          }
+        }
+      } else {
+        modTime = entry.getDateModified();
+      }
+    } catch (IOException e) {
+      LOG.error("Cannot get mount point", e);
+    }
+    return modTime;
+  }
+}


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


[37/50] [abbrv] hadoop git commit: YARN-8649. NPE in localizer hearbeat processing if a container is killed while localizing. Contributed by lujie

Posted by sh...@apache.org.
YARN-8649. NPE in localizer hearbeat processing if a container is killed while localizing. Contributed by lujie


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

Branch: refs/heads/HDFS-12943
Commit: 585ebd873a55bedd2a364d256837f08ada8ba032
Parents: bed8cb6
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Aug 23 09:29:46 2018 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Aug 23 09:29:46 2018 -0500

----------------------------------------------------------------------
 .../localizer/LocalResourcesTrackerImpl.java            |  5 +++++
 .../localizer/ResourceLocalizationService.java          | 12 ++++++++----
 .../localizer/TestResourceLocalizationService.java      | 12 +++++++++++-
 3 files changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
index dd31543..ad24c62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourcesTrackerImpl.java
@@ -500,6 +500,11 @@ class LocalResourcesTrackerImpl implements LocalResourcesTracker {
 
     Path localPath = new Path(rPath, req.getPath().getName());
     LocalizedResource rsrc = localrsrc.get(req);
+    if (rsrc == null) {
+      LOG.warn("Resource " + req + " has been removed"
+          + " and will no longer be localized");
+      return null;
+    }
     rsrc.setLocalPath(localPath);
     LocalResource lr = LocalResource.newInstance(req.getResource(),
         req.getType(), req.getVisibility(), req.getSize(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 2892d1f..d9b887f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -885,6 +885,9 @@ public class ResourceLocalizationService extends CompositeService
             Path publicDirDestPath =
                 publicRsrc.getPathForLocalization(key, publicRootPath,
                     delService);
+            if (publicDirDestPath == null) {
+              return;
+            }
             if (!publicDirDestPath.getParent().equals(publicRootPath)) {
               createParentDirs(publicDirDestPath, publicRootPath);
               if (diskValidator != null) {
@@ -1175,10 +1178,11 @@ public class ResourceLocalizationService extends CompositeService
           LocalResourcesTracker tracker = getLocalResourcesTracker(
               next.getVisibility(), user, applicationId);
           if (tracker != null) {
-            ResourceLocalizationSpec resource =
-                NodeManagerBuilderUtils.newResourceLocalizationSpec(next,
-                getPathForLocalization(next, tracker));
-            rsrcs.add(resource);
+            Path localPath = getPathForLocalization(next, tracker);
+            if (localPath != null) {
+              rsrcs.add(NodeManagerBuilderUtils.newResourceLocalizationSpec(
+                  next, localPath));
+            }
           }
         } catch (IOException e) {
           LOG.error("local path for PRIVATE localization could not be " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/585ebd87/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index 2b9148e..21896ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -1717,8 +1717,18 @@ public class TestResourceLocalizationService {
       assertEquals("NM should tell localizer to be LIVE in Heartbeat.",
           LocalizerAction.LIVE, response.getLocalizerAction());
 
-      // Cleanup application.
+      // Cleanup container.
       spyService.handle(new ContainerLocalizationCleanupEvent(c, rsrcs));
+      dispatcher.await();
+      try {
+        /*Directly send heartbeat to introduce race as container
+          is being cleaned up.*/
+        locRunnerForContainer.processHeartbeat(
+              Collections.singletonList(rsrcSuccess));
+      } catch (Exception e) {
+        fail("Exception should not have been thrown on processing heartbeat");
+      }
+      // Cleanup application.
       spyService.handle(new ApplicationLocalizationEvent(
           LocalizationEventType.DESTROY_APPLICATION_RESOURCES, app));
       dispatcher.await();


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


[25/50] [abbrv] hadoop git commit: YARN-8683. Support to display pending scheduling requests in RM app attempt page. Contributed by Tao Yang.

Posted by sh...@apache.org.
YARN-8683. Support to display pending scheduling requests in RM app attempt page. Contributed by Tao Yang.


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

Branch: refs/heads/HDFS-12943
Commit: 54d0bf8935e35aad0f4d67df358ceb970cfcd713
Parents: d3fef7a
Author: Weiwei Yang <ww...@apache.org>
Authored: Tue Aug 21 19:00:31 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Tue Aug 21 19:00:31 2018 +0800

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        | 10 ++++
 .../scheduler/AppSchedulingInfo.java            | 17 +++++++
 .../scheduler/YarnScheduler.java                | 10 ++++
 .../placement/AppPlacementAllocator.java        |  6 +++
 .../LocalityAppPlacementAllocator.java          |  5 ++
 .../SingleConstraintAppPlacementAllocator.java  |  4 +-
 .../webapp/RMAppAttemptBlock.java               | 24 +++++++--
 .../resourcemanager/webapp/dao/AppInfo.java     | 10 ++++
 .../webapp/dao/ResourceRequestInfo.java         | 52 ++++++++++++++++++--
 9 files changed, 129 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index d2e81a5..9d2b058 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -878,6 +878,16 @@ public abstract class AbstractYarnScheduler
   }
 
   @Override
+  public List<SchedulingRequest> getPendingSchedulingRequestsForAttempt(
+      ApplicationAttemptId attemptId) {
+    SchedulerApplicationAttempt attempt = getApplicationAttempt(attemptId);
+    if (attempt != null) {
+      return attempt.getAppSchedulingInfo().getAllSchedulingRequests();
+    }
+    return null;
+  }
+
+  @Override
   public Priority checkAndGetApplicationPriority(
       Priority priorityRequestedByApp, UserGroupInformation user,
       String queueName, ApplicationId applicationId) throws YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 8074f06..d63d2b82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -476,6 +476,23 @@ public class AppSchedulingInfo {
     return ret;
   }
 
+  /**
+   * Fetch SchedulingRequests.
+   * @return All pending SchedulingRequests.
+   */
+  public List<SchedulingRequest> getAllSchedulingRequests() {
+    List<SchedulingRequest> ret = new ArrayList<>();
+    try {
+      this.readLock.lock();
+      schedulerKeyToAppPlacementAllocator.values().stream()
+          .filter(ap -> ap.getSchedulingRequest() != null)
+          .forEach(ap -> ret.add(ap.getSchedulingRequest()));
+    } finally {
+      this.readLock.unlock();
+    }
+    return ret;
+  }
+
   public PendingAsk getNextPendingAsk() {
     try {
       readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index 43d55c4..0f7a5b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -364,6 +364,16 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
       ApplicationAttemptId attemptId);
 
   /**
+   * Get pending scheduling request for specified application attempt.
+   *
+   * @param attemptId the id of the application attempt
+   *
+   * @return pending scheduling requests
+   */
+  List<SchedulingRequest> getPendingSchedulingRequestsForAttempt(
+      ApplicationAttemptId attemptId);
+
+  /**
    * Get cluster max priority.
    * 
    * @return maximum priority of cluster

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
index df58157..088b3dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/AppPlacementAllocator.java
@@ -192,4 +192,10 @@ public abstract class AppPlacementAllocator<N extends SchedulerNode> {
     this.rmContext = rmContext;
     this.schedulerRequestKey = schedulerRequestKey;
   }
+
+  /**
+   * Get pending SchedulingRequest.
+   * @return SchedulingRequest
+   */
+  public abstract SchedulingRequest getSchedulingRequest();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
index e1239a9..f1df343 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalityAppPlacementAllocator.java
@@ -425,4 +425,9 @@ public class LocalityAppPlacementAllocator <N extends SchedulerNode>
       writeLock.unlock();
     }
   }
+
+  @Override
+  public SchedulingRequest getSchedulingRequest() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
index 2b610f2..914f35d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SingleConstraintAppPlacementAllocator.java
@@ -505,8 +505,8 @@ public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
     }
   }
 
-  @VisibleForTesting
-  SchedulingRequest getSchedulingRequest() {
+  @Override
+  public SchedulingRequest getSchedulingRequest() {
     return schedulingRequest;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
index 43a6ac9..99a569a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppAttemptBlock.java
@@ -87,10 +87,14 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         .h3("Total Outstanding Resource Requests: "
             + getTotalResource(resourceRequests))
         .table("#resourceRequests").thead().tr().th(".priority", "Priority")
+        .th(".allocationRequestId", "AllocationRequestId")
         .th(".resource", "ResourceName").th(".capacity", "Capability")
         .th(".containers", "NumContainers")
         .th(".relaxlocality", "RelaxLocality")
-        .th(".labelexpression", "NodeLabelExpression").__().__().tbody();
+        .th(".labelexpression", "NodeLabelExpression")
+        .th(".executiontype", "ExecutionType")
+        .th(".allocationTags", "AllocationTags")
+        .th(".placementConstraint", "PlacementConstraint").__().__().tbody();
 
     StringBuilder resourceRequestTableData = new StringBuilder("[\n");
     for (ResourceRequestInfo resourceRequest  : resourceRequests) {
@@ -99,7 +103,11 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       }
       resourceRequestTableData.append("[\"")
           .append(String.valueOf(resourceRequest.getPriority())).append("\",\"")
-          .append(resourceRequest.getResourceName()).append("\",\"")
+          .append(String.valueOf(resourceRequest.getAllocationRequestId()))
+          .append("\",\"")
+          .append(resourceRequest.getResourceName() == null ? "N/A"
+              : resourceRequest.getResourceName())
+          .append("\",\"")
           .append(StringEscapeUtils.escapeEcmaScript(StringEscapeUtils
               .escapeHtml4(String.valueOf(resourceRequest.getCapability()))))
           .append("\",\"")
@@ -109,6 +117,15 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
           .append("\",\"")
           .append(resourceRequest.getNodeLabelExpression() == null ? "N/A"
               : resourceRequest.getNodeLabelExpression())
+          .append("\",\"")
+          .append(resourceRequest.getExecutionTypeRequest() == null ? "N/A"
+              : resourceRequest.getExecutionTypeRequest().getExecutionType())
+          .append("\",\"")
+          .append(resourceRequest.getAllocationTags() == null ? "N/A" :
+              StringUtils.join(resourceRequest.getAllocationTags(), ","))
+          .append("\",\"")
+          .append(resourceRequest.getPlacementConstraint() == null ? "N/A"
+              : resourceRequest.getPlacementConstraint())
           .append("\"],\n");
     }
     if (resourceRequestTableData
@@ -132,7 +149,8 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       if (request.getNumContainers() == 0) {
         continue;
       }
-      if (request.getResourceName().equals(ResourceRequest.ANY)) {
+      if (request.getResourceName() == null || request.getResourceName()
+          .equals(ResourceRequest.ANY)) {
         Resources.addTo(
             totalResource,
             Resources.multiply(request.getCapability().getResource(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
index 9d82bc7..63b6fe0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -230,6 +231,15 @@ public class AppInfo {
                 resourceRequests.add(new ResourceRequestInfo(req));
               }
             }
+
+            List<SchedulingRequest> schedulingRequestsRaw = rm.getRMContext()
+                .getScheduler().getPendingSchedulingRequestsForAttempt(
+                    attempt.getAppAttemptId());
+            if (schedulingRequestsRaw != null) {
+              for (SchedulingRequest req : schedulingRequestsRaw) {
+                resourceRequests.add(new ResourceRequestInfo(req));
+              }
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54d0bf89/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.java
index 030af45..beab9d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceRequestInfo.java
@@ -21,11 +21,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
+import java.util.Set;
 
 /**
  * Simple class representing a resource request.
@@ -36,6 +38,8 @@ public class ResourceRequestInfo {
 
   @XmlElement(name = "priority")
   private int priority;
+  @XmlElement(name = "allocationRequestId")
+  private long allocationRequestId;
   @XmlElement(name = "resourceName")
   private String resourceName;
   @XmlElement(name = "capability")
@@ -50,16 +54,17 @@ public class ResourceRequestInfo {
   @XmlElement(name = "executionTypeRequest")
   private ExecutionTypeRequestInfo executionTypeRequest;
 
-  @XmlElement(name = "executionType")
-  private String executionType;
-  @XmlElement(name = "enforceExecutionType")
-  private boolean enforceExecutionType;
+  @XmlElement(name = "placementConstraint")
+  private String placementConstraint;
+  @XmlElement(name = "allocationTags")
+  private Set<String> allocationTags;
 
   public ResourceRequestInfo() {
   }
 
   public ResourceRequestInfo(ResourceRequest request) {
     priority = request.getPriority().getPriority();
+    allocationRequestId = request.getAllocationRequestId();
     resourceName = request.getResourceName();
     capability = new ResourceInfo(request.getCapability());
     numContainers = request.getNumContainers();
@@ -71,6 +76,21 @@ public class ResourceRequestInfo {
     }
   }
 
+  public ResourceRequestInfo(SchedulingRequest request) {
+    priority = request.getPriority().getPriority();
+    allocationRequestId = request.getAllocationRequestId();
+    capability = new ResourceInfo(request.getResourceSizing().getResources());
+    numContainers = request.getResourceSizing().getNumAllocations();
+    if (request.getExecutionType() != null) {
+      executionTypeRequest =
+          new ExecutionTypeRequestInfo(request.getExecutionType());
+    }
+    allocationTags = request.getAllocationTags();
+    if (request.getPlacementConstraint() != null) {
+      placementConstraint = request.getPlacementConstraint().toString();
+    }
+  }
+
   public Priority getPriority() {
     return Priority.newInstance(priority);
   }
@@ -128,4 +148,28 @@ public class ResourceRequestInfo {
   public ExecutionTypeRequestInfo getExecutionTypeRequest() {
     return executionTypeRequest;
   }
+
+  public String getPlacementConstraint() {
+    return placementConstraint;
+  }
+
+  public void setPlacementConstraint(String placementConstraint) {
+    this.placementConstraint = placementConstraint;
+  }
+
+  public Set<String> getAllocationTags() {
+    return allocationTags;
+  }
+
+  public void setAllocationTags(Set<String> allocationTags) {
+    this.allocationTags = allocationTags;
+  }
+
+  public long getAllocationRequestId() {
+    return allocationRequestId;
+  }
+
+  public void setAllocationRequestId(long allocationRequestId) {
+    this.allocationRequestId = allocationRequestId;
+  }
 }


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


[03/50] [abbrv] hadoop git commit: HDFS-13747. Statistic for list_located_status is incremented incorrectly by listStatusIterator. Contributed by Antal Mihalyi.

Posted by sh...@apache.org.
HDFS-13747. Statistic for list_located_status is incremented incorrectly by listStatusIterator. Contributed by Antal Mihalyi.


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

Branch: refs/heads/HDFS-12943
Commit: c67b0650ea10896c6289703595faef0d262c00b3
Parents: 1697a02
Author: Xiao Chen <xi...@apache.org>
Authored: Thu Aug 16 23:13:10 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Thu Aug 16 23:14:21 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DistributedFileSystem.java    | 6 +++++-
 .../org/apache/hadoop/hdfs/TestDistributedFileSystem.java     | 7 +++++++
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c67b0650/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 70b3679..28c1e27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -1217,7 +1217,11 @@ public class DistributedFileSystem extends FileSystem
       thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
           needLocation);
       statistics.incrementReadOps(1);
-      storageStatistics.incrementOpCounter(OpType.LIST_LOCATED_STATUS);
+      if (needLocation) {
+        storageStatistics.incrementOpCounter(OpType.LIST_LOCATED_STATUS);
+      } else {
+        storageStatistics.incrementOpCounter(OpType.LIST_STATUS);
+      }
       if (thisListing == null) { // the directory does not exist
         throw new FileNotFoundException("File " + p + " does not exist.");
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c67b0650/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index f09255e..46323dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -706,6 +706,7 @@ public class TestDistributedFileSystem {
       // Iterative ls test
       long mkdirOp = getOpStatistics(OpType.MKDIRS);
       long listStatusOp = getOpStatistics(OpType.LIST_STATUS);
+      long locatedListStatusOP = getOpStatistics(OpType.LIST_LOCATED_STATUS);
       for (int i = 0; i < 10; i++) {
         Path p = new Path(dir, Integer.toString(i));
         fs.mkdirs(p);
@@ -729,6 +730,12 @@ public class TestDistributedFileSystem {
         checkStatistics(fs, readOps, ++writeOps, largeReadOps);
         checkOpStatistics(OpType.MKDIRS, mkdirOp);
         checkOpStatistics(OpType.LIST_STATUS, listStatusOp);
+
+        fs.listLocatedStatus(dir);
+        locatedListStatusOP++;
+        readOps++;
+        checkStatistics(fs, readOps, writeOps, largeReadOps);
+        checkOpStatistics(OpType.LIST_LOCATED_STATUS, locatedListStatusOP);
       }
       
       opCount = getOpStatistics(OpType.GET_STATUS);


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