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 in...@apache.org on 2017/07/28 16:50:10 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-14557. Document HADOOP-8143 (Change distcp to have -pb on by default). Contributed by Bharat Viswanadham. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-10467 b85bcc8c9 -> a60031c7f (forced update)


HADOOP-14557. Document HADOOP-8143 (Change distcp to have -pb on by default). Contributed by Bharat Viswanadham.


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

Branch: refs/heads/HDFS-10467
Commit: 44350fdf495f5cf1bb15b1fe6f6e9587d3de0a59
Parents: 8c2c812
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Jul 20 18:23:13 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Jul 20 18:23:13 2017 -0700

----------------------------------------------------------------------
 hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44350fdf/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
index ee0a93e..925b24e 100644
--- a/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
+++ b/hadoop-tools/hadoop-distcp/src/site/markdown/DistCp.md.vm
@@ -217,7 +217,7 @@ Command Line Options
 
 Flag              | Description                          | Notes
 ----------------- | ------------------------------------ | --------
-`-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded.
+`-p[rbugpcaxt]` | Preserve r: replication number b: block size u: user g: group p: permission c: checksum-type a: ACL x: XAttr t: timestamp | When `-update` is specified, status updates will **not** be synchronized unless the file sizes also differ (i.e. unless the file is re-created). If -pa is specified, DistCp preserves the permissions also because ACLs are a super-set of permissions. The option -pr is only valid if both source and target directory are not erasure coded. **Note:** If -p option's are not specified, then by default block size is preserved.
 `-i` | Ignore failures | As explained in the Appendix, this option will keep more accurate statistics about the copy than the default case. It also preserves logs from failed copies, which can be valuable for debugging. Finally, a failing map will not cause the job to fail before all splits are attempted.
 `-log <logdir>` | Write logs to \<logdir\> | DistCp keeps logs of each file it attempts to copy as map output. If a map fails, the log output will not be retained if it is re-executed.
 `-m <num_maps>` | Maximum number of simultaneous copies | Specify the number of maps to copy data. Note that more maps may not necessarily improve throughput.


---------------------------------------------------------------------
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: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: a60031c7f86b584f0d5472a142fbb41df416f649
Parents: 13f300e
Author: Inigo Goiri <in...@apache.org>
Authored: Thu May 11 09:57:03 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 28 09:48:38 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   38 +
 .../resolver/FederationNamespaceInfo.java       |   46 +-
 .../federation/resolver/RemoteLocation.java     |   46 +-
 .../federation/router/ConnectionContext.java    |  104 +
 .../federation/router/ConnectionManager.java    |  408 ++++
 .../federation/router/ConnectionPool.java       |  314 +++
 .../federation/router/ConnectionPoolId.java     |  117 ++
 .../router/RemoteLocationContext.java           |   38 +-
 .../server/federation/router/RemoteMethod.java  |  164 ++
 .../server/federation/router/RemoteParam.java   |   71 +
 .../hdfs/server/federation/router/Router.java   |   58 +-
 .../federation/router/RouterRpcClient.java      |  856 ++++++++
 .../federation/router/RouterRpcServer.java      | 1867 +++++++++++++++++-
 .../src/main/resources/hdfs-default.xml         |   95 +
 .../server/federation/FederationTestUtils.java  |   80 +-
 .../hdfs/server/federation/MockResolver.java    |   90 +-
 .../server/federation/RouterConfigBuilder.java  |   20 +-
 .../server/federation/RouterDFSCluster.java     |  535 +++--
 .../server/federation/router/TestRouter.java    |   31 +-
 .../server/federation/router/TestRouterRpc.java |  869 ++++++++
 .../router/TestRouterRpcMultiDestination.java   |  216 ++
 21 files changed, 5675 insertions(+), 388 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 97a6e59..c15b576 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1098,6 +1098,44 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // HDFS Router-based federation
   public static final String FEDERATION_ROUTER_PREFIX =
       "dfs.federation.router.";
+  public static final String DFS_ROUTER_DEFAULT_NAMESERVICE =
+      FEDERATION_ROUTER_PREFIX + "default.nameserviceId";
+  public static final String DFS_ROUTER_HANDLER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.count";
+  public static final int DFS_ROUTER_HANDLER_COUNT_DEFAULT = 10;
+  public static final String DFS_ROUTER_READER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.queue.size";
+  public static final int DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_READER_COUNT_KEY =
+      FEDERATION_ROUTER_PREFIX + "reader.count";
+  public static final int DFS_ROUTER_READER_COUNT_DEFAULT = 1;
+  public static final String DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY =
+      FEDERATION_ROUTER_PREFIX + "handler.queue.size";
+  public static final int DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT = 100;
+  public static final String DFS_ROUTER_RPC_BIND_HOST_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-bind-host";
+  public static final int DFS_ROUTER_RPC_PORT_DEFAULT = 8888;
+  public static final String DFS_ROUTER_RPC_ADDRESS_KEY =
+      FEDERATION_ROUTER_PREFIX + "rpc-address";
+  public static final String DFS_ROUTER_RPC_ADDRESS_DEFAULT =
+      "0.0.0.0:" + DFS_ROUTER_RPC_PORT_DEFAULT;
+  public static final String DFS_ROUTER_RPC_ENABLE =
+      FEDERATION_ROUTER_PREFIX + "rpc.enable";
+  public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true;
+
+  // HDFS Router NN client
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
+      FEDERATION_ROUTER_PREFIX + "connection.pool-size";
+  public static final int DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT =
+      64;
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN =
+      FEDERATION_ROUTER_PREFIX + "connection.pool.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+  public static final String DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS =
+      FEDERATION_ROUTER_PREFIX + "connection.clean.ms";
+  public static final long DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT =
+      TimeUnit.SECONDS.toMillis(10);
 
   // HDFS Router State Store connection
   public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
index bbaeca3..edcd308 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
@@ -23,15 +23,14 @@ import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
  * Represents information about a single nameservice/namespace in a federated
  * HDFS cluster.
  */
-public class FederationNamespaceInfo
-    implements Comparable<FederationNamespaceInfo>, RemoteLocationContext {
+public class FederationNamespaceInfo extends RemoteLocationContext {
 
   /** Block pool identifier. */
-  private String blockPoolId;
+  private final String blockPoolId;
   /** Cluster identifier. */
-  private String clusterId;
+  private final String clusterId;
   /** Nameservice identifier. */
-  private String nameserviceId;
+  private final String nameserviceId;
 
   public FederationNamespaceInfo(String bpId, String clId, String nsId) {
     this.blockPoolId = bpId;
@@ -39,15 +38,16 @@ public class FederationNamespaceInfo
     this.nameserviceId = nsId;
   }
 
-  /**
-   * The HDFS nameservice id for this namespace.
-   *
-   * @return Nameservice identifier.
-   */
+  @Override
   public String getNameserviceId() {
     return this.nameserviceId;
   }
 
+  @Override
+  public String getDest() {
+    return this.nameserviceId;
+  }
+
   /**
    * The HDFS cluster id for this namespace.
    *
@@ -67,33 +67,7 @@ public class FederationNamespaceInfo
   }
 
   @Override
-  public int hashCode() {
-    return this.nameserviceId.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null) {
-      return false;
-    } else if (obj instanceof FederationNamespaceInfo) {
-      return this.compareTo((FederationNamespaceInfo) obj) == 0;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int compareTo(FederationNamespaceInfo info) {
-    return this.nameserviceId.compareTo(info.getNameserviceId());
-  }
-
-  @Override
   public String toString() {
     return this.nameserviceId + "->" + this.blockPoolId + ":" + this.clusterId;
   }
-
-  @Override
-  public String getDest() {
-    return this.nameserviceId;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
index eef136d..6aa12ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
@@ -17,34 +17,51 @@
  */
 package org.apache.hadoop.hdfs.server.federation.resolver;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
 
 /**
  * A single in a remote namespace consisting of a nameservice ID
  * and a HDFS path.
  */
-public class RemoteLocation implements RemoteLocationContext {
+public class RemoteLocation extends RemoteLocationContext {
 
   /** Identifier of the remote namespace for this location. */
-  private String nameserviceId;
+  private final String nameserviceId;
+  /** Identifier of the namenode in the namespace for this location. */
+  private final String namenodeId;
   /** Path in the remote location. */
-  private String path;
+  private final String path;
 
   /**
    * Create a new remote location.
    *
+   * @param nsId
+   * @param pPath
+   */
+  public RemoteLocation(String nsId, String pPath) {
+    this(nsId, null, pPath);
+  }
+
+  /**
+   * Create a new remote location pointing to a particular namenode in the
+   * namespace.
+   *
    * @param nsId Destination namespace.
    * @param pPath Path in the destination namespace.
    */
-  public RemoteLocation(String nsId, String pPath) {
+  public RemoteLocation(String nsId, String nnId, String pPath) {
     this.nameserviceId = nsId;
+    this.namenodeId = nnId;
     this.path = pPath;
   }
 
   @Override
   public String getNameserviceId() {
-    return this.nameserviceId;
+    String ret = this.nameserviceId;
+    if (this.namenodeId != null) {
+      ret += "-" + this.namenodeId;
+    }
+    return ret;
   }
 
   @Override
@@ -54,21 +71,6 @@ public class RemoteLocation implements RemoteLocationContext {
 
   @Override
   public String toString() {
-    return this.nameserviceId + "->" + this.path;
-  }
-
-  @Override
-  public int hashCode() {
-    return new HashCodeBuilder(17, 31)
-        .append(this.nameserviceId)
-        .append(this.path)
-        .toHashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    return (obj != null &&
-        obj.getClass() == this.getClass() &&
-        obj.hashCode() == this.hashCode());
+    return getNameserviceId() + "->" + this.path;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
new file mode 100644
index 0000000..1d27b51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.ipc.RPC;
+
+/**
+ * Context to track a connection in a {@link ConnectionPool}. When a client uses
+ * a connection, it increments a counter to mark it as active. Once the client
+ * is done with the connection, it decreases the counter. It also takes care of
+ * closing the connection once is not active.
+ */
+public class ConnectionContext {
+
+  /** Client for the connection. */
+  private final ProxyAndInfo<ClientProtocol> client;
+  /** How many threads are using this connection. */
+  private int numThreads = 0;
+  /** If the connection is closed. */
+  private boolean closed = false;
+
+
+  public ConnectionContext(ProxyAndInfo<ClientProtocol> connection) {
+    this.client = connection;
+  }
+
+  /**
+   * Check if the connection is active.
+   *
+   * @return True if the connection is active.
+   */
+  public synchronized boolean isActive() {
+    return this.numThreads > 0;
+  }
+
+  /**
+   * Check if the connection is closed.
+   *
+   * @return If the connection is closed.
+   */
+  public synchronized boolean isClosed() {
+    return this.closed;
+  }
+
+  /**
+   * Check if the connection can be used. It checks if the connection is used by
+   * another thread or already closed.
+   *
+   * @return True if the connection can be used.
+   */
+  public synchronized boolean isUsable() {
+    return !isActive() && !isClosed();
+  }
+
+  /**
+   * Get the connection client.
+   *
+   * @return Connection client.
+   */
+  public synchronized ProxyAndInfo<ClientProtocol> getClient() {
+    this.numThreads++;
+    return this.client;
+  }
+
+  /**
+   * Release this connection. If the connection was closed, close the proxy.
+   * Otherwise, mark the connection as not used by us anymore.
+   */
+  public synchronized void release() {
+    if (--this.numThreads == 0 && this.closed) {
+      close();
+    }
+  }
+
+  /**
+   * We will not use this connection anymore. If it's not being used, we close
+   * it. Otherwise, we let release() do it once we are done with it.
+   */
+  public synchronized void close() {
+    this.closed = true;
+    if (this.numThreads == 0) {
+      ClientProtocol proxy = this.client.getProxy();
+      // Nobody should be using this anymore so it should close right away
+      RPC.stopProxy(proxy);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
new file mode 100644
index 0000000..d93d498
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
@@ -0,0 +1,408 @@
+/**
+ * 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 java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements a pool of connections for the {@link Router} to be able to open
+ * many connections to many Namenodes.
+ */
+public class ConnectionManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionManager.class);
+
+  /** Number of parallel new connections to create. */
+  protected static final int MAX_NEW_CONNECTIONS = 100;
+
+  /** Minimum amount of active connections: 50%. */
+  protected static final float MIN_ACTIVE_RATIO = 0.5f;
+
+
+  /** Configuration for the connection manager, pool and sockets. */
+  private final Configuration conf;
+
+  /** Min number of connections per user + nn. */
+  private final int minSize = 1;
+  /** Max number of connections per user + nn. */
+  private final int maxSize;
+
+  /** How often we close a pool for a particular user + nn. */
+  private final long poolCleanupPeriodMs;
+  /** How often we close a connection in a pool. */
+  private final long connectionCleanupPeriodMs;
+
+  /** Map of connection pools, one pool per user + NN. */
+  private final Map<ConnectionPoolId, ConnectionPool> pools;
+  /** Lock for accessing pools. */
+  private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+  private final Lock readLock = readWriteLock.readLock();
+  private final Lock writeLock = readWriteLock.writeLock();
+
+  /** Queue for creating new connections. */
+  private final BlockingQueue<ConnectionPool> creatorQueue =
+      new ArrayBlockingQueue<>(MAX_NEW_CONNECTIONS);
+  /** Create new connections asynchronously. */
+  private final ConnectionCreator creator;
+  /** Periodic executor to remove stale connection pools. */
+  private final ScheduledThreadPoolExecutor cleaner =
+      new ScheduledThreadPoolExecutor(1);
+
+  /** If the connection manager is running. */
+  private boolean running = false;
+
+
+  /**
+   * Creates a proxy client connection pool manager.
+   *
+   * @param config Configuration for the connections.
+   * @param minPoolSize Min size of the connection pool.
+   * @param maxPoolSize Max size of the connection pool.
+   */
+  public ConnectionManager(Configuration config) {
+    this.conf = config;
+
+    // Configure minimum and maximum connection pools
+    this.maxSize = this.conf.getInt(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT);
+
+    // Map with the connections indexed by UGI and Namenode
+    this.pools = new HashMap<>();
+
+    // Create connections in a thread asynchronously
+    this.creator = new ConnectionCreator(creatorQueue);
+    this.creator.setDaemon(true);
+
+    // Cleanup periods
+    this.poolCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_CLEAN_DEFAULT);
+    LOG.info("Cleaning connection pools every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.poolCleanupPeriodMs));
+    this.connectionCleanupPeriodMs = this.conf.getLong(
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS,
+        DFSConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CLEAN_MS_DEFAULT);
+    LOG.info("Cleaning connections every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(this.connectionCleanupPeriodMs));
+  }
+
+  /**
+   * Start the connection manager.
+   */
+  public void start() {
+    // Start the thread that creates connections asynchronously
+    this.creator.start();
+
+    // Schedule a task to remove stale connection pools and sockets
+    long recyleTimeMs = Math.min(
+        poolCleanupPeriodMs, connectionCleanupPeriodMs);
+    LOG.info("Cleaning every {} seconds",
+        TimeUnit.MILLISECONDS.toSeconds(recyleTimeMs));
+    this.cleaner.scheduleAtFixedRate(
+        new CleanupTask(), 0, recyleTimeMs, TimeUnit.MILLISECONDS);
+
+    // Mark the manager as running
+    this.running = true;
+  }
+
+  /**
+   * Stop the connection manager by closing all the pools.
+   */
+  public void close() {
+    this.creator.shutdown();
+    this.cleaner.shutdown();
+    this.running = false;
+
+    writeLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        pool.close();
+      }
+      this.pools.clear();
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Fetches the next available proxy client in the pool. Each client connection
+   * is reserved for a single user and cannot be reused until free.
+   *
+   * @param ugi User group information.
+   * @param nnAddress Namenode address for the connection.
+   * @return Proxy client to connect to nnId as UGI.
+   * @throws IOException If the connection cannot be obtained.
+   */
+  public ConnectionContext getConnection(
+      UserGroupInformation ugi, String nnAddress) throws IOException {
+
+    // Check if the manager is shutdown
+    if (!this.running) {
+      LOG.error(
+          "Cannot get a connection to {} because the manager isn't running",
+          nnAddress);
+      return null;
+    }
+
+    // Try to get the pool if created
+    ConnectionPoolId connectionId = new ConnectionPoolId(ugi, nnAddress);
+    ConnectionPool pool = null;
+    readLock.lock();
+    try {
+      pool = this.pools.get(connectionId);
+    } finally {
+      readLock.unlock();
+    }
+
+    // Create the pool if not created before
+    if (pool == null) {
+      writeLock.lock();
+      try {
+        pool = this.pools.get(connectionId);
+        if (pool == null) {
+          pool = new ConnectionPool(
+              this.conf, nnAddress, ugi, this.minSize, this.maxSize);
+          this.pools.put(connectionId, pool);
+        }
+      } finally {
+        writeLock.unlock();
+      }
+    }
+
+    ConnectionContext conn = pool.getConnection();
+
+    // Add a new connection to the pool if it wasn't usable
+    if (conn == null || !conn.isUsable()) {
+      if (!this.creatorQueue.offer(pool)) {
+        LOG.error("Cannot add more than {} connections at the same time",
+            MAX_NEW_CONNECTIONS);
+      }
+    }
+
+    if (conn != null && conn.isClosed()) {
+      LOG.error("We got a closed connection from {}", pool);
+      conn = null;
+    }
+
+    return conn;
+  }
+
+  /**
+   * Get the number of connection pools.
+   *
+   * @return Number of connection pools.
+   */
+  public int getNumConnectionPools() {
+    readLock.lock();
+    try {
+      return pools.size();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get number of open connections.
+   *
+   * @return Number of open connections.
+   */
+  public int getNumConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get number of active connections.
+   *
+   * @return Number of active connections.
+   */
+  public int getNumActiveConnections() {
+    int total = 0;
+    readLock.lock();
+    try {
+      for (ConnectionPool pool : this.pools.values()) {
+        total += pool.getNumActiveConnections();
+      }
+    } finally {
+      readLock.unlock();
+    }
+    return total;
+  }
+
+  /**
+   * Get the number of connections to be created.
+   *
+   * @return Number of connections to be created.
+   */
+  public int getNumCreatingConnections() {
+    return this.creatorQueue.size();
+  }
+
+  /**
+   * Removes stale connections not accessed recently from the pool. This is
+   * invoked periodically.
+   */
+  private class CleanupTask implements Runnable {
+
+    @Override
+    public void run() {
+      long currentTime = Time.now();
+      List<ConnectionPoolId> toRemove = new LinkedList<>();
+
+      // Look for stale pools
+      readLock.lock();
+      try {
+        for (Entry<ConnectionPoolId, ConnectionPool> entry : pools.entrySet()) {
+          ConnectionPool pool = entry.getValue();
+          long lastTimeActive = pool.getLastActiveTime();
+          boolean isStale =
+              currentTime > (lastTimeActive + poolCleanupPeriodMs);
+          if (lastTimeActive > 0 && isStale) {
+            // Remove this pool
+            LOG.debug("Closing and removing stale pool {}", pool);
+            pool.close();
+            ConnectionPoolId poolId = entry.getKey();
+            toRemove.add(poolId);
+          } else {
+            // Keep this pool but clean connections inside
+            LOG.debug("Cleaning up {}", pool);
+            cleanup(pool);
+          }
+        }
+      } finally {
+        readLock.unlock();
+      }
+
+      // Remove stale pools
+      if (!toRemove.isEmpty()) {
+        writeLock.lock();
+        try {
+          for (ConnectionPoolId poolId : toRemove) {
+            pools.remove(poolId);
+          }
+        } finally {
+          writeLock.unlock();
+        }
+      }
+    }
+
+    /**
+     * Clean the unused connections for this pool.
+     *
+     * @param pool Connection pool to cleanup.
+     */
+    private void cleanup(ConnectionPool pool) {
+      if (pool.getNumConnections() > pool.getMinSize()) {
+        // Check if the pool hasn't been active in a while or not 50% are used
+        long timeSinceLastActive = Time.now() - pool.getLastActiveTime();
+        int total = pool.getNumConnections();
+        int active = getNumActiveConnections();
+        if (timeSinceLastActive > connectionCleanupPeriodMs ||
+            active < MIN_ACTIVE_RATIO * total) {
+          // Remove and close 1 connection
+          List<ConnectionContext> conns = pool.removeConnections(1);
+          for (ConnectionContext conn : conns) {
+            conn.close();
+          }
+          LOG.debug("Removed connection {} used {} seconds ago. " +
+              "Pool has {}/{} connections", pool.getConnectionPoolId(),
+              TimeUnit.MILLISECONDS.toSeconds(timeSinceLastActive),
+              pool.getNumConnections(), pool.getMaxSize());
+        }
+      }
+    }
+  }
+
+  /**
+   * Thread that creates connections asynchronously.
+   */
+  private static class ConnectionCreator extends Thread {
+    /** If the creator is running. */
+    private boolean running = true;
+    /** Queue to push work to. */
+    private BlockingQueue<ConnectionPool> queue;
+
+    ConnectionCreator(BlockingQueue<ConnectionPool> blockingQueue) {
+      super("Connection creator");
+      this.queue = blockingQueue;
+    }
+
+    @Override
+    public void run() {
+      while (this.running) {
+        try {
+          ConnectionPool pool = this.queue.take();
+          try {
+            int total = pool.getNumConnections();
+            int active = pool.getNumActiveConnections();
+            if (pool.getNumConnections() < pool.getMaxSize() &&
+                active >= MIN_ACTIVE_RATIO * total) {
+              ConnectionContext conn = pool.newConnection();
+              pool.addConnection(conn);
+            } else {
+              LOG.debug("Cannot add more than {} connections to {}",
+                  pool.getMaxSize(), pool);
+            }
+          } catch (IOException e) {
+            LOG.error("Cannot create a new connection", e);
+          }
+        } catch (InterruptedException e) {
+          LOG.error("The connection creator was interrupted");
+          this.running = false;
+        }
+      }
+    }
+
+    /**
+     * Stop this connection creator.
+     */
+    public void shutdown() {
+      this.running = false;
+      this.interrupt();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
new file mode 100644
index 0000000..f76f621
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
@@ -0,0 +1,314 @@
+/**
+ * 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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Maintains a pool of connections for each User (including tokens) + NN. The
+ * RPC client maintains a single socket, to achieve throughput similar to a NN,
+ * each request is multiplexed across multiple sockets/connections from a
+ * pool.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPool {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConnectionPool.class);
+
+
+  /** Configuration settings for the connection pool. */
+  private final Configuration conf;
+
+  /** Identifier for this connection pool. */
+  private final ConnectionPoolId connectionPoolId;
+  /** Namenode this pool connects to. */
+  private final String namenodeAddress;
+  /** User for this connections. */
+  private final UserGroupInformation ugi;
+
+  /** Pool of connections. We mimic a COW array. */
+  private volatile List<ConnectionContext> connections = new ArrayList<>();
+  /** Connection index for round-robin. */
+  private final AtomicInteger clientIndex = new AtomicInteger(0);
+
+  /** Min number of connections per user. */
+  private final int minSize;
+  /** Max number of connections per user. */
+  private final int maxSize;
+
+  /** The last time a connection was active. */
+  private volatile long lastActiveTime = 0;
+
+
+  protected ConnectionPool(Configuration config, String address,
+      UserGroupInformation user, int minPoolSize, int maxPoolSize)
+          throws IOException {
+
+    this.conf = config;
+
+    // Connection pool target
+    this.ugi = user;
+    this.namenodeAddress = address;
+    this.connectionPoolId =
+        new ConnectionPoolId(this.ugi, this.namenodeAddress);
+
+    // Set configuration parameters for the pool
+    this.minSize = minPoolSize;
+    this.maxSize = maxPoolSize;
+
+    // Add minimum connections to the pool
+    for (int i=0; i<this.minSize; i++) {
+      ConnectionContext newConnection = newConnection();
+      this.connections.add(newConnection);
+    }
+    LOG.debug("Created connection pool \"{}\" with {} connections",
+        this.connectionPoolId, this.minSize);
+  }
+
+  /**
+   * Get the maximum number of connections allowed in this pool.
+   *
+   * @return Maximum number of connections.
+   */
+  protected int getMaxSize() {
+    return this.maxSize;
+  }
+
+  /**
+   * Get the minimum number of connections in this pool.
+   *
+   * @return Minimum number of connections.
+   */
+  protected int getMinSize() {
+    return this.minSize;
+  }
+
+  /**
+   * Get the connection pool identifier.
+   *
+   * @return Connection pool identifier.
+   */
+  protected ConnectionPoolId getConnectionPoolId() {
+    return this.connectionPoolId;
+  }
+
+  /**
+   * Return the next connection round-robin.
+   *
+   * @return Connection context.
+   */
+  protected ConnectionContext getConnection() {
+
+    this.lastActiveTime = Time.now();
+
+    // Get a connection from the pool following round-robin
+    ConnectionContext conn = null;
+    List<ConnectionContext> tmpConnections = this.connections;
+    int size = tmpConnections.size();
+    int threadIndex = this.clientIndex.getAndIncrement();
+    for (int i=0; i<size; i++) {
+      int index = (threadIndex + i) % size;
+      conn = tmpConnections.get(index);
+      if (conn != null && !conn.isUsable()) {
+        return conn;
+      }
+    }
+
+    // We return a connection even if it's active
+    return conn;
+  }
+
+  /**
+   * Add a connection to the current pool. It uses a Copy-On-Write approach.
+   *
+   * @param conns New connections to add to the pool.
+   */
+  public synchronized void addConnection(ConnectionContext conn) {
+    List<ConnectionContext> tmpConnections = new ArrayList<>(this.connections);
+    tmpConnections.add(conn);
+    this.connections = tmpConnections;
+
+    this.lastActiveTime = Time.now();
+  }
+
+  /**
+   * Remove connections from the current pool.
+   *
+   * @param num Number of connections to remove.
+   * @return Removed connections.
+   */
+  public synchronized List<ConnectionContext> removeConnections(int num) {
+    List<ConnectionContext> removed = new LinkedList<>();
+
+    // Remove and close the last connection
+    List<ConnectionContext> tmpConnections = new ArrayList<>();
+    for (int i=0; i<this.connections.size(); i++) {
+      ConnectionContext conn = this.connections.get(i);
+      if (i < this.minSize || i < this.connections.size() - num) {
+        tmpConnections.add(conn);
+      } else {
+        removed.add(conn);
+      }
+    }
+    this.connections = tmpConnections;
+
+    return removed;
+  }
+
+  /**
+   * Close the connection pool.
+   */
+  protected synchronized void close() {
+    long timeSinceLastActive = TimeUnit.MILLISECONDS.toSeconds(
+        Time.now() - getLastActiveTime());
+    LOG.debug("Shutting down connection pool \"{}\" used {} seconds ago",
+        this.connectionPoolId, timeSinceLastActive);
+
+    for (ConnectionContext connection : this.connections) {
+      connection.close();
+    }
+    this.connections.clear();
+  }
+
+  /**
+   * Number of connections in the pool.
+   *
+   * @return Number of connections.
+   */
+  protected int getNumConnections() {
+    return this.connections.size();
+  }
+
+  /**
+   * Number of active connections in the pool.
+   *
+   * @return Number of active connections.
+   */
+  protected int getNumActiveConnections() {
+    int ret = 0;
+
+    List<ConnectionContext> tmpConnections = this.connections;
+    for (ConnectionContext conn : tmpConnections) {
+      if (conn.isActive()) {
+        ret++;
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * Get the last time the connection pool was used.
+   *
+   * @return Last time the connection pool was used.
+   */
+  protected long getLastActiveTime() {
+    return this.lastActiveTime;
+  }
+
+  @Override
+  public String toString() {
+    return this.connectionPoolId.toString();
+  }
+
+  /**
+   * Create a new proxy wrapper for a client NN connection.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException
+   */
+  public ConnectionContext newConnection() throws IOException {
+    return newConnection(this.conf, this.namenodeAddress, this.ugi);
+  }
+
+  /**
+   * Creates a proxy wrapper for a client NN connection. Each proxy contains
+   * context for a single user/security context. To maximize throughput it is
+   * recommended to use multiple connection per user+server, allowing multiple
+   * writes and reads to be dispatched in parallel.
+   *
+   * @param conf Configuration for the connection.
+   * @param nnAddress Address of server supporting the ClientProtocol.
+   * @param ugi User context.
+   * @return Proxy for the target ClientProtocol that contains the user's
+   *         security context.
+   * @throws IOException If it cannot be created.
+   */
+  protected static ConnectionContext newConnection(Configuration conf,
+      String nnAddress, UserGroupInformation ugi)
+          throws IOException {
+    RPC.setProtocolEngine(
+        conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class);
+
+    final RetryPolicy defaultPolicy = RetryUtils.getDefaultRetryPolicy(
+        conf,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
+        HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
+        HdfsConstants.SAFEMODE_EXCEPTION_CLASS_NAME);
+
+    SocketFactory factory = SocketFactory.getDefault();
+    if (UserGroupInformation.isSecurityEnabled()) {
+      SaslRpcServer.init(conf);
+    }
+    InetSocketAddress socket = NetUtils.createSocketAddr(nnAddress);
+    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
+    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
+        ClientNamenodeProtocolPB.class, version, socket, ugi, conf,
+        factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
+    ClientProtocol client = new ClientNamenodeProtocolTranslatorPB(proxy);
+    Text dtService = SecurityUtil.buildTokenService(socket);
+
+    ProxyAndInfo<ClientProtocol> clientProxy =
+        new ProxyAndInfo<ClientProtocol>(client, dtService, socket);
+    ConnectionContext connection = new ConnectionContext(clientProxy);
+    return connection;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.java
new file mode 100644
index 0000000..a3a78de
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPoolId.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.hdfs.server.federation.router;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * Identifier for a connection for a user to a namenode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ConnectionPoolId implements Comparable<ConnectionPoolId> {
+
+  /** Namenode identifier. */
+  private final String nnId;
+  /** Information about the user. */
+  private final UserGroupInformation ugi;
+
+  /**
+   * New connection pool identifier.
+   *
+   * @param ugi Information of the user issuing the request.
+   * @param nnId Namenode address with port.
+   */
+  public ConnectionPoolId(final UserGroupInformation ugi, final String nnId) {
+    this.nnId = nnId;
+    this.ugi = ugi;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = new HashCodeBuilder(17, 31)
+        .append(this.nnId)
+        .append(this.ugi.toString())
+        .append(this.getTokenIds())
+        .toHashCode();
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ConnectionPoolId) {
+      ConnectionPoolId other = (ConnectionPoolId) o;
+      if (!this.nnId.equals(other.nnId)) {
+        return false;
+      }
+      if (!this.ugi.toString().equals(other.ugi.toString())) {
+        return false;
+      }
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      return thisTokens.equals(otherTokens);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return this.ugi + " " + this.getTokenIds() + "->" + this.nnId;
+  }
+
+  @Override
+  public int compareTo(ConnectionPoolId other) {
+    int ret = this.nnId.compareTo(other.nnId);
+    if (ret == 0) {
+      ret = this.ugi.toString().compareTo(other.ugi.toString());
+    }
+    if (ret == 0) {
+      String thisTokens = this.getTokenIds().toString();
+      String otherTokens = other.getTokenIds().toString();
+      ret = thisTokens.compareTo(otherTokens);
+    }
+    return ret;
+  }
+
+  /**
+   * Get the token identifiers for this connection.
+   * @return List with the token identifiers.
+   */
+  private List<String> getTokenIds() {
+    List<String> tokenIds = new ArrayList<>();
+    Collection<Token<? extends TokenIdentifier>> tokens = this.ugi.getTokens();
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      byte[] tokenIdBytes = token.getIdentifier();
+      String tokenId = Arrays.toString(tokenIdBytes);
+      tokenIds.add(tokenId);
+    }
+    Collections.sort(tokenIds);
+    return tokenIds;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
index da6066b..a90c460 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
@@ -17,22 +17,52 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
 /**
- * Interface for objects that are unique to a namespace.
+ * Base class for objects that are unique to a namespace.
  */
-public interface RemoteLocationContext {
+public abstract class RemoteLocationContext
+    implements Comparable<RemoteLocationContext> {
 
   /**
    * Returns an identifier for a unique namespace.
    *
    * @return Namespace identifier.
    */
-  String getNameserviceId();
+  public abstract String getNameserviceId();
 
   /**
    * Destination in this location. For example the path in a remote namespace.
    *
    * @return Destination in this location.
    */
-  String getDest();
+  public abstract String getDest();
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(getNameserviceId())
+        .append(getDest())
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RemoteLocationContext) {
+      RemoteLocationContext other = (RemoteLocationContext) obj;
+      return this.getNameserviceId().equals(other.getNameserviceId()) &&
+          this.getDest().equals(other.getDest());
+    }
+    return false;
+  }
+
+  @Override
+  public int compareTo(RemoteLocationContext info) {
+    int ret = this.getNameserviceId().compareTo(info.getNameserviceId());
+    if (ret == 0) {
+      ret = this.getDest().compareTo(info.getDest());
+    }
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
new file mode 100644
index 0000000..cd57d45
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java
@@ -0,0 +1,164 @@
+/**
+ * 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 java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Determines the remote client protocol method and the parameter list for a
+ * specific location.
+ */
+public class RemoteMethod {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteMethod.class);
+
+
+  /** List of parameters: static and dynamic values, matchings types. */
+  private final Object[] params;
+  /** List of method parameters types, matches parameters. */
+  private final Class<?>[] types;
+  /** String name of the ClientProtocol method. */
+  private final String methodName;
+
+  /**
+   * Create a method with no parameters.
+   *
+   * @param method The string name of the ClientProtocol method.
+   */
+  public RemoteMethod(String method) {
+    this.params = null;
+    this.types = null;
+    this.methodName = method;
+  }
+
+  /**
+   * Creates a remote method generator.
+   *
+   * @param method The string name of the ClientProtocol method.
+   * @param pTypes A list of types to use to locate the specific method.
+   * @param pParams A list of parameters for the method. The order of the
+   *          parameter list must match the order and number of the types.
+   *          Parameters are grouped into 2 categories:
+   *          <ul>
+   *          <li>Static parameters that are immutable across locations.
+   *          <li>Dynamic parameters that are determined for each location by a
+   *          RemoteParam object. To specify a dynamic parameter, pass an
+   *          instance of RemoteParam in place of the parameter value.
+   *          </ul>
+   * @throws IOException If the types and parameter lists are not valid.
+   */
+  public RemoteMethod(String method, Class<?>[] pTypes, Object... pParams)
+      throws IOException {
+
+    if (pParams.length != pTypes.length) {
+      throw new IOException("Invalid parameters for method " + method);
+    }
+
+    this.params = pParams;
+    this.types = pTypes;
+    this.methodName = method;
+  }
+
+  /**
+   * Get the represented java method.
+   *
+   * @return Method
+   * @throws IOException If the method cannot be found.
+   */
+  public Method getMethod() throws IOException {
+    try {
+      if (types != null) {
+        return ClientProtocol.class.getDeclaredMethod(methodName, types);
+      } else {
+        return ClientProtocol.class.getDeclaredMethod(methodName);
+      }
+    } catch (NoSuchMethodException e) {
+      // Re-throw as an IOException
+      LOG.error("Cannot get method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    } catch (SecurityException e) {
+      LOG.error("Cannot access method {} with types {}",
+          methodName, Arrays.toString(types), e);
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Get the calling types for this method.
+   *
+   * @return An array of calling types.
+   */
+  public Class<?>[] getTypes() {
+    return this.types;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location using no context.
+   *
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams() {
+    return this.getParams(null);
+  }
+
+  /**
+   * Get the name of the method.
+   *
+   * @return Name of the method.
+   */
+  public String getMethodName() {
+    return this.methodName;
+  }
+
+  /**
+   * Generate a list of parameters for this specific location. Parameters are
+   * grouped into 2 categories:
+   * <ul>
+   * <li>Static parameters that are immutable across locations.
+   * <li>Dynamic parameters that are determined for each location by a
+   * RemoteParam object.
+   * </ul>
+   *
+   * @param context The context identifying the location.
+   * @return A list of parameters for the method customized for the location.
+   */
+  public Object[] getParams(RemoteLocationContext context) {
+    if (this.params == null) {
+      return new Object[] {};
+    }
+    Object[] objList = new Object[this.params.length];
+    for (int i = 0; i < this.params.length; i++) {
+      Object currentObj = this.params[i];
+      if (currentObj instanceof RemoteParam) {
+        // Map the parameter using the context
+        RemoteParam paramGetter = (RemoteParam) currentObj;
+        objList[i] = paramGetter.getParameterForContext(context);
+      } else {
+        objList[i] = currentObj;
+      }
+    }
+    return objList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
new file mode 100644
index 0000000..8816ff6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteParam.java
@@ -0,0 +1,71 @@
+/**
+ * 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 java.util.Map;
+
+/**
+ * A dynamically assignable parameter that is location-specific.
+ * <p>
+ * There are 2 ways this mapping is determined:
+ * <ul>
+ * <li>Default: Uses the RemoteLocationContext's destination
+ * <li>Map: Uses the value of the RemoteLocationContext key provided in the
+ * parameter map.
+ * </ul>
+ */
+public class RemoteParam {
+
+  private final Map<? extends Object, ? extends Object> paramMap;
+
+  /**
+   * Constructs a default remote parameter. Always maps the value to the
+   * destination of the provided RemoveLocationContext.
+   */
+  public RemoteParam() {
+    this.paramMap = null;
+  }
+
+  /**
+   * Constructs a map based remote parameter. Determines the value using the
+   * provided RemoteLocationContext as a key into the map.
+   *
+   * @param map Map with RemoteLocationContext keys.
+   */
+  public RemoteParam(
+      Map<? extends RemoteLocationContext, ? extends Object> map) {
+    this.paramMap = map;
+  }
+
+  /**
+   * Determine the appropriate value for this parameter based on the location.
+   *
+   * @param context Context identifying the location.
+   * @return A parameter specific to this location.
+   */
+  public Object getParameterForContext(RemoteLocationContext context) {
+    if (context == null) {
+      return null;
+    } else if (this.paramMap != null) {
+      return this.paramMap.get(context);
+    } else {
+      // Default case
+      return context.getDest();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
index fe0d02a..019a5cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -22,12 +22,14 @@ import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.new
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
@@ -36,6 +38,8 @@ import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Router that provides a unified view of multiple federated HDFS clusters. It
@@ -60,7 +64,7 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceStability.Evolving
 public class Router extends CompositeService {
 
-  private static final Log LOG = LogFactory.getLog(Router.class);
+  private static final Logger LOG = LoggerFactory.getLogger(Router.class);
 
 
   /** Configuration for the Router. */
@@ -71,6 +75,7 @@ public class Router extends CompositeService {
 
   /** RPC interface to the client. */
   private RouterRpcServer rpcServer;
+  private InetSocketAddress rpcAddress;
 
   /** Interface with the State Store. */
   private StateStoreService stateStore;
@@ -105,9 +110,6 @@ public class Router extends CompositeService {
   protected void serviceInit(Configuration configuration) throws Exception {
     this.conf = configuration;
 
-    // TODO Interface to the State Store
-    this.stateStore = null;
-
     // Resolver to track active NNs
     this.namenodeResolver = newActiveNamenodeResolver(
         this.conf, this.stateStore);
@@ -122,6 +124,15 @@ public class Router extends CompositeService {
       throw new IOException("Cannot find subcluster resolver");
     }
 
+    if (conf.getBoolean(
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE,
+        DFSConfigKeys.DFS_ROUTER_RPC_ENABLE_DEFAULT)) {
+      // Create RPC server
+      this.rpcServer = createRpcServer();
+      addService(this.rpcServer);
+      this.setRpcServerAddress(rpcServer.getRpcAddress());
+    }
+
     super.serviceInit(conf);
   }
 
@@ -171,11 +182,13 @@ public class Router extends CompositeService {
       router.init(conf);
       router.start();
     } catch (Throwable e) {
-      LOG.error("Failed to start router.", e);
+      LOG.error("Failed to start router", e);
       terminate(1, e);
     }
   }
 
+
+
   /////////////////////////////////////////////////////////
   // RPC Server
   /////////////////////////////////////////////////////////
@@ -183,7 +196,7 @@ public class Router extends CompositeService {
   /**
    * Create a new Router RPC server to proxy ClientProtocol requests.
    *
-   * @return RouterRpcServer
+   * @return New Router RPC Server.
    * @throws IOException If the router RPC server was not started.
    */
   protected RouterRpcServer createRpcServer() throws IOException {
@@ -200,6 +213,35 @@ public class Router extends CompositeService {
     return this.rpcServer;
   }
 
+  /**
+   * Set the current RPC socket for the router.
+   *
+   * @param rpcAddress RPC address.
+   */
+  protected void setRpcServerAddress(InetSocketAddress address) {
+    this.rpcAddress = address;
+
+    // Use the RPC address as our unique router Id
+    if (this.rpcAddress != null) {
+      try {
+        String hostname = InetAddress.getLocalHost().getHostName();
+        setRouterId(hostname + ":" + this.rpcAddress.getPort());
+      } catch (UnknownHostException ex) {
+        LOG.error("Cannot set unique router ID, address not resolvable {}",
+            this.rpcAddress);
+      }
+    }
+  }
+
+  /**
+   * Get the current RPC socket address for the router.
+   *
+   * @return InetSocketAddress
+   */
+  public InetSocketAddress getRpcServerAddress() {
+    return this.rpcAddress;
+  }
+
   /////////////////////////////////////////////////////////
   // Submodule getters
   /////////////////////////////////////////////////////////


---------------------------------------------------------------------
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: YARN-6804. [YARN core changes] Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi

Posted by in...@apache.org.
YARN-6804. [YARN core changes] Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi


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

Branch: refs/heads/HDFS-10467
Commit: 4a771d9010de0867ac901bead075383ddf1f30dc
Parents: 8e3a992
Author: Jian He <ji...@apache.org>
Authored: Fri Jul 21 11:31:33 2017 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Jul 21 11:31:33 2017 -0700

----------------------------------------------------------------------
 .../client/binding/RegistryPathUtils.java       |  2 +-
 .../hadoop/registry/client/types/Endpoint.java  |  4 +-
 .../registry/client/types/ServiceRecord.java    |  4 +-
 .../hadoop-yarn-server-nodemanager/pom.xml      |  4 ++
 .../runtime/DockerLinuxContainerRuntime.java    | 67 +++++++++++++++-----
 .../linux/runtime/docker/DockerRunCommand.java  |  6 ++
 .../impl/container-executor.c                   |  4 ++
 .../test/test-container-executor.c              | 16 ++---
 .../runtime/TestDockerContainerRuntime.java     | 58 +++++++++++++----
 9 files changed, 121 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
index 5d8ea3f..5fa45f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
@@ -213,6 +213,6 @@ public class RegistryPathUtils {
    * @return a string suitable for use in registry paths.
    */
   public static String encodeYarnID(String yarnId) {
-    return yarnId.replace("_", "-");
+    return yarnId.replace("container", "ctr").replace("_", "-");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
index 395f836..392884f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -46,7 +46,7 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public final class Endpoint implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
index 674d6d3..d40866a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonAnyGetter;
 import com.fasterxml.jackson.annotation.JsonAnySetter;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -37,7 +37,7 @@ import java.util.Map;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public class ServiceRecord implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index a0f4ef7..094519a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -52,6 +52,10 @@
       <artifactId>hadoop-yarn-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+    </dependency>
+    <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.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/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 8db03bc..e058d6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
@@ -101,6 +102,11 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  *     property.
  *   </li>
  *   <li>
+ *     {@code YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME} sets the
+ *     hostname to be used by the Docker container. If not specified, a
+ *     hostname will be derived from the container ID.
+ *   </li>
+ *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER}
  *     controls whether the Docker container is a privileged container. In order
  *     to use privileged containers, the
@@ -134,6 +140,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       "^(([a-zA-Z0-9.-]+)(:\\d+)?/)?([a-z0-9_./-]+)(:[\\w.-]+)?$";
   private static final Pattern dockerImagePattern =
       Pattern.compile(DOCKER_IMAGE_PATTERN);
+  public static final String HOSTNAME_PATTERN =
+      "^[a-zA-Z0-9][a-zA-Z0-9_.-]+$";
+  private static final Pattern hostnamePattern = Pattern.compile(
+      HOSTNAME_PATTERN);
 
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_IMAGE =
@@ -147,6 +157,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_NETWORK =
       "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
+  @InterfaceAudience.Private
+  public static final String ENV_DOCKER_CONTAINER_HOSTNAME =
+      "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME";
+  @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER =
       "YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER";
   @InterfaceAudience.Private
@@ -211,9 +225,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     this.privilegedOperationExecutor = privilegedOperationExecutor;
 
     if (cGroupsHandler == null) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("cGroupsHandler is null - cgroups not in use.");
-      }
+      LOG.info("cGroupsHandler is null - cgroups not in use.");
     } else {
       this.cGroupsHandler = cGroupsHandler;
     }
@@ -267,6 +279,29 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     throw new ContainerExecutionException(msg);
   }
 
+  public static void validateHostname(String hostname) throws
+      ContainerExecutionException {
+    if (hostname != null && !hostname.isEmpty()) {
+      if (!hostnamePattern.matcher(hostname).matches()) {
+        throw new ContainerExecutionException("Hostname '" + hostname
+            + "' doesn't match docker hostname pattern");
+      }
+    }
+  }
+
+  /** Set a DNS friendly hostname. */
+  private void setHostname(DockerRunCommand runCommand, String
+      containerIdStr, String name)
+      throws ContainerExecutionException {
+    if (name == null || name.isEmpty()) {
+      name = RegistryPathUtils.encodeYarnID(containerIdStr);
+      validateHostname(name);
+    }
+
+    LOG.info("setting hostname in container to: " + name);
+    runCommand.setHostname(name);
+  }
+
   /**
    * If CGROUPS in enabled and not set to none, then set the CGROUP parent for
    * the command instance.
@@ -343,10 +378,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       return false;
     }
 
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Privileged container requested for : " + container
-          .getContainerId().toString());
-    }
+    LOG.info("Privileged container requested for : " + container
+        .getContainerId().toString());
 
     //Ok, so we have been asked to run a privileged container. Security
     // checks need to be run. Each violation is an error.
@@ -375,10 +408,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throw new ContainerExecutionException(message);
     }
 
-    if (LOG.isInfoEnabled()) {
-      LOG.info("All checks pass. Launching privileged container for : "
-          + container.getContainerId().toString());
-    }
+    LOG.info("All checks pass. Launching privileged container for : "
+        + container.getContainerId().toString());
 
     return true;
   }
@@ -413,6 +444,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         .getEnvironment();
     String imageName = environment.get(ENV_DOCKER_CONTAINER_IMAGE);
     String network = environment.get(ENV_DOCKER_CONTAINER_NETWORK);
+    String hostname = environment.get(ENV_DOCKER_CONTAINER_HOSTNAME);
 
     if(network == null || network.isEmpty()) {
       network = defaultNetwork;
@@ -420,6 +452,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
 
     validateContainerNetworkType(network);
 
+    validateHostname(hostname);
+
     validateImageName(imageName);
 
     String containerIdStr = container.getContainerId().toString();
@@ -450,12 +484,13 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         runAsUser, imageName)
         .detachOnRun()
         .setContainerWorkDir(containerWorkDir.toString())
-        .setNetworkType(network)
-        .setCapabilities(capabilities)
+        .setNetworkType(network);
+    setHostname(runCommand, containerIdStr, hostname);
+    runCommand.setCapabilities(capabilities)
         .addMountLocation(CGROUPS_ROOT_DIRECTORY,
             CGROUPS_ROOT_DIRECTORY + ":ro", false);
-    List<String> allDirs = new ArrayList<>(containerLocalDirs);
 
+    List<String> allDirs = new ArrayList<>(containerLocalDirs);
     allDirs.addAll(filecacheDirs);
     allDirs.add(containerWorkDir.toString());
     allDirs.addAll(containerLogDirs);
@@ -493,9 +528,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         ENV_DOCKER_CONTAINER_RUN_OVERRIDE_DISABLE);
 
     if (disableOverride != null && disableOverride.equals("true")) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("command override disabled");
-      }
+      LOG.info("command override disabled");
     } else {
       List<String> overrideCommands = new ArrayList<>();
       Path launchDst =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.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/linux/runtime/docker/DockerRunCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
index f79f4ed..b645754 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
@@ -91,6 +91,12 @@ public class DockerRunCommand extends DockerCommand {
 
     return this;
   }
+
+  public DockerRunCommand setHostname(String hostname) {
+    super.addCommandArguments("--hostname=" + hostname);
+    return this;
+  }
+
   public DockerRunCommand addDevice(String sourceDevice, String
       destinationDevice) {
     super.addCommandArguments("--device=" + sourceDevice + ":" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 5d138f3..5070d62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1215,6 +1215,7 @@ char* sanitize_docker_command(const char *line) {
     {"rm", no_argument, 0, 'r' },
     {"workdir", required_argument, 0, 'w' },
     {"net", required_argument, 0, 'e' },
+    {"hostname", required_argument, 0, 'h' },
     {"cgroup-parent", required_argument, 0, 'g' },
     {"privileged", no_argument, 0, 'p' },
     {"cap-add", required_argument, 0, 'a' },
@@ -1256,6 +1257,9 @@ char* sanitize_docker_command(const char *line) {
       case 'e':
         quote_and_append_arg(&output, &output_size, "--net=", optarg);
         break;
+      case 'h':
+        quote_and_append_arg(&output, &output_size, "--hostname=", optarg);
+        break;
       case 'v':
         quote_and_append_arg(&output, &output_size, "-v ", optarg);
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index 83d11ec..b7d0e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -1088,17 +1088,17 @@ void test_trim_function() {
 void test_sanitize_docker_command() {
 
   char *input[] = {
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
     "run ''''''''"
   };
   char *expected_output[] = {
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
       "run ''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"'' ",
   };
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4a771d90/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.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/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index f611843..9894dcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -69,6 +70,7 @@ public class TestDockerContainerRuntime {
   private PrivilegedOperationExecutor mockExecutor;
   private CGroupsHandler mockCGroupsHandler;
   private String containerId;
+  private String defaultHostname;
   private Container container;
   private ContainerId cId;
   private ContainerLaunchContext context;
@@ -108,6 +110,7 @@ public class TestDockerContainerRuntime {
         .mock(PrivilegedOperationExecutor.class);
     mockCGroupsHandler = Mockito.mock(CGroupsHandler.class);
     containerId = "container_id";
+    defaultHostname = RegistryPathUtils.encodeYarnID(containerId);
     container = mock(Container.class);
     cId = mock(ContainerId.class);
     context = mock(ContainerLaunchContext.class);
@@ -287,6 +290,7 @@ public class TestDockerContainerRuntime {
         .append("--user=%2$s -d ")
         .append("--workdir=%3$s ")
         .append("--net=host ")
+        .append("--hostname=" + defaultHostname + " ")
         .append(getExpectedTestCapabilitiesArgumentString())
         .append(getExpectedCGroupsMountString())
         .append("-v %4$s:%4$s ")
@@ -365,7 +369,7 @@ public class TestDockerContainerRuntime {
     String disallowedNetwork = "sdn" + Integer.toString(randEngine.nextInt());
 
     try {
-      env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+      env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
           disallowedNetwork);
       runtime.launchContainer(builder.build());
       Assert.fail("Network was expected to be disallowed: " +
@@ -378,8 +382,11 @@ public class TestDockerContainerRuntime {
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS.length;
     String allowedNetwork = YarnConfiguration
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS[randEngine.nextInt(size)];
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         allowedNetwork);
+    String expectedHostname = "test.hostname";
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_HOSTNAME,
+        expectedHostname);
 
     //this should cause no failures.
 
@@ -393,6 +400,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + allowedNetwork + " ")
+            .append("--hostname=" + expectedHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -448,6 +456,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork1 + " ")
+            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -471,7 +480,7 @@ public class TestDockerContainerRuntime {
     //now set an explicit (non-default) allowedNetwork and ensure that it is
     // used.
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         customNetwork2);
     runtime.launchContainer(builder.build());
 
@@ -485,6 +494,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork2 + " ")
+            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -505,7 +515,7 @@ public class TestDockerContainerRuntime {
 
     //disallowed network should trigger a launch failure
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         customNetwork3);
     try {
       runtime.launchContainer(builder.build());
@@ -524,8 +534,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "invalid-value");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
     runtime.launchContainer(builder.build());
 
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -552,8 +562,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -575,8 +585,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     //By default
     // yarn.nodemanager.runtime.linux.docker.privileged-containers.acl
     // is empty. So we expect this launch to fail.
@@ -605,8 +615,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -632,8 +642,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     runtime.launchContainer(builder.build());
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -927,4 +937,24 @@ public class TestDockerContainerRuntime {
       }
     }
   }
+
+  @Test
+  public void testDockerHostnamePattern() throws Exception {
+    String[] validNames = {"ab", "a.b.c.d", "a1-b.cd.ef", "0AB.", "C_D-"};
+
+    String[] invalidNames = {"a", "a#.b.c", "-a.b.c", "a@b.c", "a/b/c"};
+
+    for (String name : validNames) {
+      DockerLinuxContainerRuntime.validateHostname(name);
+    }
+
+    for (String name : invalidNames) {
+      try {
+        DockerLinuxContainerRuntime.validateHostname(name);
+        Assert.fail(name + " is an invalid hostname and should fail the regex");
+      } catch (ContainerExecutionException ce) {
+        continue;
+      }
+    }
+  }
 }


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletGen.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletGen.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletGen.java
new file mode 100644
index 0000000..c6ca93c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletGen.java
@@ -0,0 +1,449 @@
+/**
+* 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.webapp.hamlet2;
+
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.webapp.WebAppException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Generates a specific hamlet implementation class from a spec class
+ * using a generic hamlet implementation class.
+ */
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+public class HamletGen {
+  static final Logger LOG = LoggerFactory.getLogger(HamletGen.class);
+  static final Options opts = new Options();
+  static {
+    opts.addOption("h", "help", false, "Print this help message").
+         addOption("s", "spec-class", true,
+                   "The class that holds the spec interfaces. e.g. HamletSpec").
+         addOption("i", "impl-class", true,
+                   "An implementation class. e.g. HamletImpl").
+         addOption("o", "output-class", true, "Output class name").
+         addOption("p", "output-package", true, "Output package name");
+  };
+
+  static final Pattern elementRegex = Pattern.compile("^[A-Z][A-Z0-9]*$");
+
+  int bytes = 0;
+  PrintWriter out;
+  final Set<String> endTagOptional = Sets.newHashSet();
+  final Set<String> inlineElements = Sets.newHashSet();
+  Class<?> top; // html top-level interface
+  String hamlet; // output class simple name;
+  boolean topMode;
+
+  /**
+   * Generate a specific Hamlet implementation from a spec.
+   * @param specClass holds hamlet interfaces. e.g. {@link HamletSpec}
+   * @param implClass a generic hamlet implementation. e.g. {@link HamletImpl}
+   * @param outputName name of the output class. e.g. {@link Hamlet}
+   * @param outputPkg package name of the output class.
+   * @throws IOException
+   */
+  public void generate(Class<?> specClass, Class<?> implClass,
+                       String outputName, String outputPkg) throws IOException {
+    LOG.info("Generating {} using {} and {}", new Object[]{outputName,
+             specClass, implClass});
+    out = new PrintWriter(outputName +".java", "UTF-8");
+    hamlet = basename(outputName);
+    String pkg = pkgName(outputPkg, implClass.getPackage().getName());
+    puts(0, "// Generated by HamletGen. Do NOT edit!\n",
+         "package ", pkg, ";\n",
+         "import java.io.PrintWriter;\n",
+         "import java.util.EnumSet;\n",
+         "import static java.util.EnumSet.*;\n",
+         "import static ", implClass.getName(), ".EOpt.*;\n",
+         "import org.apache.hadoop.yarn.webapp.SubView;");
+    String implClassName = implClass.getSimpleName();
+    if (!implClass.getPackage().getName().equals(pkg)) {
+      puts(0, "import ", implClass.getName(), ';');
+    }
+    puts(0, "\n",
+         "public class ", hamlet, " extends ", implClassName,
+         " implements ", specClass.getSimpleName(), "._Html {\n",
+         "  public ", hamlet, "(PrintWriter out, int nestLevel,",
+         " boolean wasInline) {\n",
+         "    super(out, nestLevel, wasInline);\n",
+         "  }\n\n", // inline is context sensitive
+         "  static EnumSet<EOpt> opt(boolean endTag, boolean inline, ",
+         "boolean pre) {\n",
+         "    EnumSet<EOpt> opts = of(ENDTAG);\n",
+         "    if (!endTag) opts.remove(ENDTAG);\n",
+         "    if (inline) opts.add(INLINE);\n",
+         "    if (pre) opts.add(PRE);\n",
+         "    return opts;\n",
+         "  }");
+    initLut(specClass);
+    genImpl(specClass, implClassName, 1);
+    LOG.info("Generating {} methods", hamlet);
+    genMethods(hamlet, top, 1);
+    puts(0, "}");
+    out.close();
+    LOG.info("Wrote {} bytes to {}.java", bytes, outputName);
+  }
+
+  String basename(String path) {
+    return path.substring(path.lastIndexOf('/') + 1);
+  }
+
+  String pkgName(String pkg, String defaultPkg) {
+    if (pkg == null || pkg.isEmpty()) return defaultPkg;
+    return pkg;
+  }
+
+  void initLut(Class<?> spec) {
+    endTagOptional.clear();
+    inlineElements.clear();
+    for (Class<?> cls : spec.getClasses()) {
+      Annotation a = cls.getAnnotation(HamletSpec.Element.class);
+      if (a != null && !((HamletSpec.Element) a).endTag()) {
+        endTagOptional.add(cls.getSimpleName());
+      }
+      if (cls.getSimpleName().equals("Inline")) {
+        for (Method method : cls.getMethods()) {
+          String retName = method.getReturnType().getSimpleName();
+          if (isElement(retName)) {
+            inlineElements.add(retName);
+          }
+        }
+      }
+    }
+  }
+
+  void genImpl(Class<?> spec, String implClassName, int indent) {
+    String specName = spec.getSimpleName();
+    for (Class<?> cls : spec.getClasses()) {
+      String className = cls.getSimpleName();
+      if (cls.isInterface()) {
+        genFactoryMethods(cls, indent);
+      }
+      if (isElement(className)) {
+        LOG.info("Generating class {}<T>", className);
+        puts(indent, "\n",
+             "public class ", className, "<T extends __>",
+             " extends EImp<T> implements ", specName, ".", className, " {\n",
+             "  public ", className, "(String name, T parent,",
+             " EnumSet<EOpt> opts) {\n",
+             "    super(name, parent, opts);\n",
+             "  }");
+        genMethods(className, cls, indent + 1);
+        puts(indent, "}");
+      } else if (className.equals("_Html")) {
+        top = cls;
+      }
+    }
+  }
+
+  void genFactoryMethods(Class<?> cls, int indent) {
+    for (Method method : cls.getDeclaredMethods()) {
+      String retName = method.getReturnType().getSimpleName();
+      String methodName = method.getName();
+      if (methodName.charAt(0) == '$') continue;
+      if (isElement(retName) && method.getParameterTypes().length == 0) {
+        genFactoryMethod(retName, methodName, indent);
+      }
+    }
+  }
+
+  void genMethods(String className, Class<?> cls, int indent) {
+    topMode = (top != null && cls.equals(top));
+    for (Method method : cls.getMethods()) {
+      String retName = method.getReturnType().getSimpleName();
+      if (method.getName().charAt(0) == '$') {
+        genAttributeMethod(className, method, indent);
+      } else if (isElement(retName)) {
+        genNewElementMethod(className, method, indent);
+      } else {
+        genCurElementMethod(className, method, indent);
+      }
+    }
+  }
+
+  void genAttributeMethod(String className, Method method, int indent) {
+    String methodName = method.getName();
+    String attrName = methodName.substring(1).replace("__", "-");
+    Type[] params = method.getGenericParameterTypes();
+    echo(indent, "\n",
+         "@Override\n",
+         "public ", className, topMode ? " " : "<T> ", methodName, "(");
+    if (params.length == 0) {
+      puts(0, ") {");
+      puts(indent,
+           "  addAttr(\"", attrName, "\", null);\n",
+           "  return this;\n", "}");
+    } else if (params.length == 1) {
+      String typeName = getTypeName(params[0]);
+      puts(0, typeName, " value) {");
+      if (typeName.equals("EnumSet<LinkType>")) {
+        puts(indent,
+             "  addRelAttr(\"", attrName, "\", value);\n",
+             "  return this;\n", "}");
+      } else if (typeName.equals("EnumSet<Media>")) {
+        puts(indent,
+             "  addMediaAttr(\"", attrName, "\", value);\n",
+             "  return this;\n", "}");
+      } else {
+        puts(indent,
+             "  addAttr(\"", attrName, "\", value);\n",
+             "  return this;\n", "}");
+      }
+    } else {
+      throwUnhandled(className, method);
+    }
+  }
+
+  String getTypeName(Type type) {
+    if (type instanceof Class<?>) {
+      return ((Class<?>)type).getSimpleName();
+    }
+    ParameterizedType pt = (ParameterizedType) type;
+    return ((Class<?>)pt.getRawType()).getSimpleName() +"<"+
+        ((Class<?>)pt.getActualTypeArguments()[0]).getSimpleName() +">";
+  }
+
+  void genFactoryMethod(String retName, String methodName, int indent) {
+    puts(indent, "\n",
+         "private <T extends __> ", retName, "<T> ", methodName,
+         "__(T e, boolean inline) {\n",
+         "  return new ", retName, "<T>(\"", StringUtils.toLowerCase(retName),
+         "\", e, opt(", !endTagOptional.contains(retName), ", inline, ",
+         retName.equals("PRE"), ")); }");
+  }
+
+  void genNewElementMethod(String className, Method method, int indent) {
+    String methodName = method.getName();
+    String retName = method.getReturnType().getSimpleName();
+    Class<?>[] params = method.getParameterTypes();
+    echo(indent, "\n",
+         "@Override\n",
+         "public ", retName, "<", className, topMode ? "> " : "<T>> ",
+         methodName, "(");
+    if (params.length == 0) {
+      puts(0, ") {");
+      puts(indent,
+           topMode ? "" : "  closeAttrs();\n",
+           "  return ", StringUtils.toLowerCase(retName), "__" + "(this, ",
+           isInline(className, retName), ");\n", "}");
+    } else if (params.length == 1) {
+      puts(0, "String selector) {");
+      puts(indent,
+           "  return setSelector(", methodName, "(), selector);\n", "}");
+    } else {
+      throwUnhandled(className, method);
+    }
+  }
+
+  boolean isInline(String container, String className) {
+    if ((container.equals("BODY") || container.equals(hamlet) ||
+         container.equals("HEAD") || container.equals("HTML")) &&
+        (className.equals("INS") || className.equals("DEL") ||
+         className.equals("SCRIPT"))) {
+      return false;
+    }
+    return inlineElements.contains(className);
+  }
+
+  void genCurElementMethod(String className, Method method, int indent) {
+    String methodName = method.getName();
+    Class<?>[] params = method.getParameterTypes();
+    if (topMode || params.length > 0) {
+      echo(indent, "\n",
+         "@Override\n",
+         "public ", className, topMode ? " " : "<T> ", methodName, "(");
+    }
+    if (params.length == 0) {
+      if (topMode) {
+        puts(0, ") {");
+        puts(indent, "  return this;\n", "}");
+      }
+    } else if (params.length == 1) {
+      if (methodName.equals("base")) {
+        puts(0, "String href) {");
+        puts(indent,
+             "  return base().$href(href).__();\n", "}");
+      } else if (methodName.equals("script")) {
+        puts(0, "String src) {");
+        puts(indent,
+             "  return setScriptSrc(script(), src).__();\n", "}");
+      } else if (methodName.equals("style")) {
+        puts(0, "Object... lines) {");
+        puts(indent,
+             "  return style().$type(\"text/css\").__(lines).__();\n", "}");
+      } else if (methodName.equals("img")) {
+        puts(0, "String src) {");
+        puts(indent,
+             "  return ", methodName, "().$src(src).__();\n", "}");
+      } else if (methodName.equals("br") || methodName.equals("hr") ||
+                 methodName.equals("col")) {
+        puts(0, "String selector) {");
+        puts(indent,
+             "  return setSelector(", methodName, "(), selector).__();\n", "}");
+      }  else if (methodName.equals("link")) {
+        puts(0, "String href) {");
+        puts(indent,
+             "  return setLinkHref(", methodName, "(), href).__();\n", "}");
+      } else if (methodName.equals("__")) {
+        if (params[0].getSimpleName().equals("Class")) {
+          puts(0, "Class<? extends SubView> cls) {");
+          puts(indent,
+               "  ", topMode ? "subView" : "_v", "(cls);\n",
+               "  return this;\n", "}");
+        } else {
+          puts(0, "Object... lines) {");
+          puts(indent,
+               "  _p(", needsEscaping(className), ", lines);\n",
+               "  return this;\n", "}");
+        }
+      } else if (methodName.equals("_r")) {
+        puts(0, "Object... lines) {");
+        puts(indent,
+             "  _p(false, lines);\n",
+             "  return this;\n", "}");
+      } else {
+        puts(0, "String cdata) {");
+        puts(indent,
+             "  return ", methodName, "().__(cdata).__();\n", "}");
+      }
+    } else if (params.length == 2) {
+      if (methodName.equals("meta")) {
+        puts(0, "String name, String content) {");
+        puts(indent,
+             "  return meta().$name(name).$content(content).__();\n", "}");
+      } else if (methodName.equals("meta_http")) {
+        puts(0, "String header, String content) {");
+        puts(indent,
+             "  return meta().$http_equiv(header).$content(content).__();\n",
+             "}");
+      } else if (methodName.equals("a")) {
+        puts(0, "String href, String anchorText) {");
+        puts(indent,
+             "  return a().$href(href).__(anchorText).__();\n", "}");
+      } else if (methodName.equals("bdo")) {
+        puts(0, "Dir dir, String cdata) {");
+        puts(indent, "  return bdo().$dir(dir).__(cdata).__();\n", "}");
+      } else if (methodName.equals("label")) {
+        puts(0, "String forId, String cdata) {");
+        puts(indent, "  return label().$for(forId).__(cdata).__();\n", "}");
+      } else if (methodName.equals("param")) {
+        puts(0, "String name, String value) {");
+        puts(indent,
+             "  return param().$name(name).$value(value).__();\n", "}");
+      } else {
+        puts(0, "String selector, String cdata) {");
+        puts(indent,
+             "  return setSelector(", methodName,
+             "(), selector).__(cdata).__();\n", "}");
+      }
+    } else if (params.length == 3) {
+      if (methodName.equals("a")) {
+        puts(0, "String selector, String href, String anchorText) {");
+        puts(indent,
+             "  return setSelector(a(), selector)",
+             ".$href(href).__(anchorText).__();\n", "}");
+      }
+    } else {
+      throwUnhandled(className, method);
+    }
+  }
+
+  static boolean needsEscaping(String eleName) {
+    return !eleName.equals("SCRIPT") && !eleName.equals("STYLE");
+  }
+
+  static void throwUnhandled(String className, Method method) {
+    throw new WebAppException("Unhandled " + className + "#" + method);
+  }
+
+  void echo(int indent, Object... args) {
+    String prev = null;
+    for (Object o : args) {
+      String s = String.valueOf(o);
+      if (!s.isEmpty() && !s.equals("\n") &&
+          (prev == null || prev.endsWith("\n"))) {
+        indent(indent);
+      }
+      prev = s;
+      out.print(s);
+      bytes += s.length();
+    }
+  }
+
+  void indent(int indent) {
+    for (int i = 0; i < indent; ++i) {
+      out.print("  ");
+      bytes += 2;
+    }
+  }
+
+  void puts(int indent, Object... args) {
+    echo(indent, args);
+    out.println();
+    ++bytes;
+  }
+
+  boolean isElement(String s) {
+    return elementRegex.matcher(s).matches();
+  }
+
+  public static void main(String[] args) throws Exception {
+    CommandLine cmd = new GnuParser().parse(opts, args);
+    if (cmd.hasOption("help")) {
+      new HelpFormatter().printHelp("Usage: hbgen [OPTIONS]", opts);
+      return;
+    }
+    // defaults
+    Class<?> specClass = HamletSpec.class;
+    Class<?> implClass = HamletImpl.class;
+    String outputClass = "HamletTmp";
+    String outputPackage = implClass.getPackage().getName();
+    if (cmd.hasOption("spec-class")) {
+      specClass = Class.forName(cmd.getOptionValue("spec-class"));
+    }
+    if (cmd.hasOption("impl-class")) {
+      implClass = Class.forName(cmd.getOptionValue("impl-class"));
+    }
+    if (cmd.hasOption("output-class")) {
+      outputClass = cmd.getOptionValue("output-class");
+    }
+    if (cmd.hasOption("output-package")) {
+      outputPackage = cmd.getOptionValue("output-package");
+    }
+    new HamletGen().generate(specClass, implClass, outputClass, outputPackage);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletImpl.java
new file mode 100644
index 0000000..995e9fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletImpl.java
@@ -0,0 +1,385 @@
+/**
+* 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.webapp.hamlet2;
+
+import com.google.common.base.Joiner;
+import static com.google.common.base.Preconditions.*;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+
+import java.io.PrintWriter;
+import java.util.EnumSet;
+import static java.util.EnumSet.*;
+import java.util.Iterator;
+
+import static org.apache.commons.lang.StringEscapeUtils.*;
+import static org.apache.hadoop.yarn.webapp.hamlet2.HamletImpl.EOpt.*;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.WebAppException;
+
+
+/**
+ * A simple unbuffered generic hamlet implementation.
+ *
+ * Zero copy but allocation on every element, which could be
+ * optimized to use a thread-local element pool.
+ *
+ * Prints HTML as it builds. So the order is important.
+ */
+@InterfaceAudience.Private
+public class HamletImpl extends HamletSpec {
+  private static final String INDENT_CHARS = "  ";
+  private static final Splitter SS = Splitter.on('.').
+      omitEmptyStrings().trimResults();
+  private static final Joiner SJ = Joiner.on(' ');
+  private static final Joiner CJ = Joiner.on(", ");
+  static final int S_ID = 0;
+  static final int S_CLASS = 1;
+
+  int nestLevel;
+  int indents; // number of indent() called. mostly for testing.
+  private final PrintWriter out;
+  private final StringBuilder sb = new StringBuilder(); // not shared
+  private boolean wasInline = false;
+
+  /**
+   * Element options. (whether it needs end tag, is inline etc.)
+   */
+  public enum EOpt {
+    /** needs end(close) tag */
+    ENDTAG,
+    /** The content is inline */
+    INLINE,
+    /** The content is preformatted */
+    PRE
+  };
+
+  /**
+   * The base class for elements
+   * @param <T> type of the parent (containing) element for the element
+   */
+  public class EImp<T extends __> implements _Child {
+    private final String name;
+    private final T parent; // short cut for parent element
+    private final EnumSet<EOpt> opts; // element options
+
+    private boolean started = false;
+    private boolean attrsClosed = false;
+
+    EImp(String name, T parent, EnumSet<EOpt> opts) {
+      this.name = name;
+      this.parent = parent;
+      this.opts = opts;
+    }
+
+    @Override
+    public T __() {
+      closeAttrs();
+      --nestLevel;
+      printEndTag(name, opts);
+      return parent;
+    }
+
+    protected void _p(boolean quote, Object... args) {
+      closeAttrs();
+      for (Object s : args) {
+        if (!opts.contains(PRE)) {
+          indent(opts);
+        }
+        out.print(quote ? escapeHtml(String.valueOf(s))
+                        : String.valueOf(s));
+        if (!opts.contains(INLINE) && !opts.contains(PRE)) {
+          out.println();
+        }
+      }
+    }
+
+    protected void _v(Class<? extends SubView> cls) {
+      closeAttrs();
+      subView(cls);
+    }
+
+    protected void closeAttrs() {
+      if (!attrsClosed) {
+        startIfNeeded();
+        ++nestLevel;
+        out.print('>');
+        if (!opts.contains(INLINE) && !opts.contains(PRE)) {
+          out.println();
+        }
+        attrsClosed = true;
+      }
+    }
+
+    protected void addAttr(String name, String value) {
+      checkState(!attrsClosed, "attribute added after content");
+      startIfNeeded();
+      printAttr(name, value);
+    }
+
+    protected void addAttr(String name, Object value) {
+      addAttr(name, String.valueOf(value));
+    }
+
+    protected void addMediaAttr(String name, EnumSet<Media> media) {
+      // 6.13 comma-separated list
+      addAttr(name, CJ.join(media));
+    }
+
+    protected void addRelAttr(String name, EnumSet<LinkType> types) {
+      // 6.12 space-separated list
+      addAttr(name, SJ.join(types));
+    }
+
+    private void startIfNeeded() {
+      if (!started) {
+        printStartTag(name, opts);
+        started = true;
+      }
+    }
+
+    protected void _inline(boolean choice) {
+      if (choice) {
+        opts.add(INLINE);
+      } else {
+        opts.remove(INLINE);
+      }
+    }
+
+    protected void _endTag(boolean choice) {
+      if (choice) {
+        opts.add(ENDTAG);
+      } else {
+        opts.remove(ENDTAG);
+      }
+    }
+
+    protected void _pre(boolean choice) {
+      if (choice) {
+        opts.add(PRE);
+      } else {
+        opts.remove(PRE);
+      }
+    }
+  }
+
+  public class Generic<T extends __> extends EImp<T> implements PCData {
+    Generic(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    public Generic<T> _inline() {
+      super._inline(true);
+      return this;
+    }
+
+    public Generic<T> _noEndTag() {
+      super._endTag(false);
+      return this;
+    }
+
+    public Generic<T> _pre() {
+      super._pre(true);
+      return this;
+    }
+
+    public Generic<T> _attr(String name, String value) {
+      addAttr(name, value);
+      return this;
+    }
+
+    public Generic<Generic<T>> _elem(String name, EnumSet<EOpt> opts) {
+      closeAttrs();
+      return new Generic<Generic<T>>(name, this, opts);
+    }
+
+    public Generic<Generic<T>> elem(String name) {
+      return _elem(name, of(ENDTAG));
+    }
+
+    @Override
+    public Generic<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public Generic<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+  }
+
+  public HamletImpl(PrintWriter out, int nestLevel, boolean wasInline) {
+    this.out = out;
+    this.nestLevel = nestLevel;
+    this.wasInline = wasInline;
+  }
+
+  public int nestLevel() {
+    return nestLevel;
+  }
+
+  public boolean wasInline() {
+    return wasInline;
+  }
+
+  public void setWasInline(boolean state) {
+    wasInline = state;
+  }
+
+  public PrintWriter getWriter() {
+    return out;
+  }
+
+  /**
+   * Create a root-level generic element.
+   * Mostly for testing purpose.
+   * @param <T> type of the parent element
+   * @param name of the element
+   * @param opts {@link EOpt element options}
+   * @return the element
+   */
+  public <T extends __>
+  Generic<T> root(String name, EnumSet<EOpt> opts) {
+    return new Generic<T>(name, null, opts);
+  }
+
+  public <T extends __> Generic<T> root(String name) {
+    return root(name, of(ENDTAG));
+  }
+
+  protected void printStartTag(String name, EnumSet<EOpt> opts) {
+    indent(opts);
+    sb.setLength(0);
+    out.print(sb.append('<').append(name).toString()); // for easier mock test
+  }
+
+  protected void indent(EnumSet<EOpt> opts) {
+    if (opts.contains(INLINE) && wasInline) {
+      return;
+    }
+    if (wasInline) {
+      out.println();
+    }
+    wasInline = opts.contains(INLINE) || opts.contains(PRE);
+    for (int i = 0; i < nestLevel; ++i) {
+      out.print(INDENT_CHARS);
+    }
+    ++indents;
+  }
+
+  protected void printEndTag(String name, EnumSet<EOpt> opts) {
+    if (!opts.contains(ENDTAG)) {
+      return;
+    }
+    if (!opts.contains(PRE)) {
+      indent(opts);
+    } else {
+      wasInline = opts.contains(INLINE);
+    }
+    sb.setLength(0);
+    out.print(sb.append("</").append(name).append('>').toString()); // ditto
+    if (!opts.contains(INLINE)) {
+      out.println();
+    }
+  }
+
+  protected void printAttr(String name, String value) {
+    sb.setLength(0);
+    sb.append(' ').append(name);
+    if (value != null) {
+      sb.append("=\"").append(escapeHtml(value)).append("\"");
+    }
+    out.print(sb.toString());
+  }
+
+  /**
+   * Sub-classes should override this to do something interesting.
+   * @param cls the sub-view class
+   */
+  protected void subView(Class<? extends SubView> cls) {
+    indent(of(ENDTAG)); // not an inline view
+    sb.setLength(0);
+    out.print(sb.append('[').append(cls.getName()).append(']').toString());
+    out.println();
+  }
+
+  /**
+   * Parse selector into id and classes
+   * @param selector in the form of (#id)?(.class)*
+   * @return an two element array [id, "space-separated classes"].
+   *         Either element could be null.
+   * @throws WebAppException when both are null or syntax error.
+   */
+  public static String[] parseSelector(String selector) {
+    String[] result = new String[]{null, null};
+    Iterable<String> rs = SS.split(selector);
+    Iterator<String> it = rs.iterator();
+    if (it.hasNext()) {
+      String maybeId = it.next();
+      if (maybeId.charAt(0) == '#') {
+        result[S_ID] = maybeId.substring(1);
+        if (it.hasNext()) {
+          result[S_CLASS] = SJ.join(Iterables.skip(rs, 1));
+        }
+      } else {
+        result[S_CLASS] = SJ.join(rs);
+      }
+      return result;
+    }
+    throw new WebAppException("Error parsing selector: "+ selector);
+  }
+
+  /**
+   * Set id and/or class attributes for an element.
+   * @param <E> type of the element
+   * @param e the element
+   * @param selector Haml form of "(#id)?(.class)*"
+   * @return the element
+   */
+  public static <E extends CoreAttrs> E setSelector(E e, String selector) {
+    String[] res = parseSelector(selector);
+    if (res[S_ID] != null) {
+      e.$id(res[S_ID]);
+    }
+    if (res[S_CLASS] != null) {
+      e.$class(res[S_CLASS]);
+    }
+    return e;
+  }
+
+  public static <E extends LINK> E setLinkHref(E e, String href) {
+    if (href.endsWith(".css")) {
+      e.$rel("stylesheet"); // required in html5
+    }
+    e.$href(href);
+    return e;
+  }
+
+  public static <E extends SCRIPT> E setScriptSrc(E e, String src) {
+    if (src.endsWith(".js")) {
+      e.$type("text/javascript"); // required in html4
+    }
+    e.$src(src);
+    return e;
+  }
+}


---------------------------------------------------------------------
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: HADOOP-14518. Customize User-Agent header sent in HTTP/HTTPS requests by WASB. Contributed by Georgi Chalakov.

Posted by in...@apache.org.
HADOOP-14518. Customize User-Agent header sent in HTTP/HTTPS requests by WASB. Contributed by Georgi Chalakov.


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

Branch: refs/heads/HDFS-10467
Commit: f2921e51f0fe613abce0a9f415a0d8ab6144aa6e
Parents: c98201b
Author: Jitendra Pandey <ji...@apache.org>
Authored: Mon Jul 24 13:59:27 2017 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Mon Jul 24 23:01:01 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |  10 +-
 .../conf/TestCommonConfigurationFields.java     |   1 +
 .../fs/azure/AzureNativeFileSystemStore.java    | 144 +++++++++++--------
 .../hadoop-azure/src/site/markdown/index.md     |  13 ++
 .../fs/azure/TestWasbUriAndConfiguration.java   |  48 +++++++
 .../src/test/resources/azure-test.xml           |   5 +
 6 files changed, 162 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/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 68b0a9d..d5ddc7f 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
@@ -499,7 +499,15 @@
     name to use for the service when the client wishes to make an RPC call.
   </description>
 </property>
-
+  <property>
+    <name>fs.azure.user.agent.prefix</name>
+    <value>unknown</value>
+    <description>
+      WASB passes User-Agent header to the Azure back-end. The default value
+      contains WASB version, Java Runtime version, Azure Client library version,
+      and the value of the configuration option fs.azure.user.agent.prefix.
+    </description>
+  </property>
 
 <property>
     <name>hadoop.security.uid.cache.secs</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 593254eb..ef74cba 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -116,6 +116,7 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPropsToSkipCompare.add("fs.azure.secure.mode");
     xmlPropsToSkipCompare.add("fs.azure.authorization");
     xmlPropsToSkipCompare.add("fs.azure.authorization.caching.enable");
+    xmlPropsToSkipCompare.add("fs.azure.user.agent.prefix");
 
     // Deprecated properties.  These should eventually be removed from the
     // class.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index 6b6f07a..7c198af 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.fs.azure.metrics.ResponseReceivedMetricUpdater;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.VersionInfo;
 import org.eclipse.jetty.util.ajax.JSON;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,6 +72,10 @@ import com.microsoft.azure.storage.StorageCredentialsAccountAndKey;
 import com.microsoft.azure.storage.StorageCredentialsSharedAccessSignature;
 import com.microsoft.azure.storage.StorageErrorCode;
 import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.Constants;
+import com.microsoft.azure.storage.StorageEvent;
+import com.microsoft.azure.storage.core.BaseRequest;
+import com.microsoft.azure.storage.SendingRequestEvent;
 import com.microsoft.azure.storage.blob.BlobListingDetails;
 import com.microsoft.azure.storage.blob.BlobProperties;
 import com.microsoft.azure.storage.blob.BlobRequestOptions;
@@ -83,13 +88,13 @@ import com.microsoft.azure.storage.core.Utility;
 
 /**
  * Core implementation of Windows Azure Filesystem for Hadoop.
- * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage 
+ * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage
  *
  */
 @InterfaceAudience.Private
 @VisibleForTesting
 public class AzureNativeFileSystemStore implements NativeFileSystemStore {
-  
+
   /**
    * Configuration knob on whether we do block-level MD5 validation on
    * upload/download.
@@ -102,6 +107,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   static final String DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME = "storageemulator";
   static final String STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.storage.emulator.account.name";
 
+  /**
+   * Configuration for User-Agent field.
+   */
+  static final String USER_AGENT_ID_KEY = "fs.azure.user.agent.prefix";
+  static final String USER_AGENT_ID_DEFAULT = "unknown";
+
   public static final Logger LOG = LoggerFactory.getLogger(AzureNativeFileSystemStore.class);
 
   private StorageInterface storageInteractionLayer;
@@ -133,15 +144,15 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private static final String KEY_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
   private static final String KEY_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
   private static final String KEY_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
-  
-  private static final String KEY_COPYBLOB_MIN_BACKOFF_INTERVAL = 
+
+  private static final String KEY_COPYBLOB_MIN_BACKOFF_INTERVAL =
     "fs.azure.io.copyblob.retry.min.backoff.interval";
-  private static final String KEY_COPYBLOB_MAX_BACKOFF_INTERVAL = 
+  private static final String KEY_COPYBLOB_MAX_BACKOFF_INTERVAL =
     "fs.azure.io.copyblob.retry.max.backoff.interval";
-  private static final String KEY_COPYBLOB_BACKOFF_INTERVAL = 
+  private static final String KEY_COPYBLOB_BACKOFF_INTERVAL =
     "fs.azure.io.copyblob.retry.backoff.interval";
-  private static final String KEY_COPYBLOB_MAX_IO_RETRIES = 
-    "fs.azure.io.copyblob.retry.max.retries";  
+  private static final String KEY_COPYBLOB_MAX_IO_RETRIES =
+    "fs.azure.io.copyblob.retry.max.retries";
 
   private static final String KEY_SELF_THROTTLE_ENABLE = "fs.azure.selfthrottling.enable";
   private static final String KEY_SELF_THROTTLE_READ_FACTOR = "fs.azure.selfthrottling.read.factor";
@@ -188,7 +199,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * The set of directories where we should store files as page blobs.
    */
   private Set<String> pageBlobDirs;
-  
+
   /**
    * Configuration key to indicate the set of directories in WASB where
    * we should do atomic folder rename synchronized with createNonRecursive.
@@ -232,11 +243,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000; // 30s
   private static final int DEFAULT_BACKOFF_INTERVAL = 1 * 1000; // 1s
   private static final int DEFAULT_MAX_RETRY_ATTEMPTS = 15;
-  
+
   private static final int DEFAULT_COPYBLOB_MIN_BACKOFF_INTERVAL = 3  * 1000;
   private static final int DEFAULT_COPYBLOB_MAX_BACKOFF_INTERVAL = 90 * 1000;
   private static final int DEFAULT_COPYBLOB_BACKOFF_INTERVAL = 30 * 1000;
-  private static final int DEFAULT_COPYBLOB_MAX_RETRY_ATTEMPTS = 15;  
+  private static final int DEFAULT_COPYBLOB_MAX_RETRY_ATTEMPTS = 15;
 
   // Self-throttling defaults. Allowed range = (0,1.0]
   // Value of 1.0 means no self-throttling.
@@ -306,6 +317,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private boolean useSecureMode = false;
   private boolean useLocalSasKeyMode = false;
 
+  // User-Agent
+  private String userAgentId;
+
   private String delegationToken;
 
   /** The error message template when container is not accessible. */
@@ -319,7 +333,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * A test hook interface that can modify the operation context we use for
    * Azure Storage operations, e.g. to inject errors.
    */
-  @VisibleForTesting 
+  @VisibleForTesting
   interface TestHookOperationContext {
     OperationContext modifyOperationContext(OperationContext original);
   }
@@ -336,11 +350,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Add a test hook to modify the operation context we use for Azure Storage
    * operations.
-   * 
+   *
    * @param testHook
    *          The test hook, or null to unset previous hooks.
    */
-  @VisibleForTesting 
+  @VisibleForTesting
   void addTestHookToOperationContext(TestHookOperationContext testHook) {
     this.testHookOperationContext = testHook;
   }
@@ -358,7 +372,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Creates a JSON serializer that can serialize a PermissionStatus object into
    * the JSON string we want in the blob metadata.
-   * 
+   *
    * @return The JSON serializer.
    */
   private static JSON createPermissionJsonSerializer() {
@@ -425,7 +439,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Check if concurrent reads and writes on the same blob are allowed.
-   * 
+   *
    * @return true if concurrent reads and OOB writes has been configured, false
    *         otherwise.
    */
@@ -437,11 +451,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * Method for the URI and configuration object necessary to create a storage
    * session with an Azure session. It parses the scheme to ensure it matches
    * the storage protocol supported by this file system.
-   * 
+   *
    * @param uri - URI for target storage blob.
    * @param conf - reference to configuration object.
    * @param instrumentation - the metrics source that will keep track of operations here.
-   * 
+   *
    * @throws IllegalArgumentException if URI or job object is null, or invalid scheme.
    */
   @Override
@@ -504,6 +518,9 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     pageBlobDirs = getDirectorySet(KEY_PAGE_BLOB_DIRECTORIES);
     LOG.debug("Page blob directories:  {}", setToString(pageBlobDirs));
 
+    // User-agent
+    userAgentId = conf.get(USER_AGENT_ID_KEY, USER_AGENT_ID_DEFAULT);
+
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
     String hbaseRoot;
@@ -539,7 +556,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Method to extract the account name from an Azure URI.
-   * 
+   *
    * @param uri
    *          -- WASB blob URI
    * @returns accountName -- the account name for the URI.
@@ -590,7 +607,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Method to extract the container name from an Azure URI.
-   * 
+   *
    * @param uri
    *          -- WASB blob URI
    * @returns containerName -- the container name for the URI. May be null.
@@ -641,7 +658,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Get the appropriate return the appropriate scheme for communicating with
    * Azure depending on whether wasb or wasbs is specified in the target URI.
-   * 
+   *
    * return scheme - HTTPS or HTTP as appropriate.
    */
   private String getHTTPScheme() {
@@ -663,7 +680,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Set the configuration parameters for this client storage session with
    * Azure.
-   * 
+   *
    * @throws AzureException
    */
   private void configureAzureStorageSession() throws AzureException {
@@ -763,10 +780,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Connect to Azure storage using anonymous credentials.
-   * 
+   *
    * @param uri
    *          - URI to target blob (R/O access to public blob)
-   * 
+   *
    * @throws StorageException
    *           raised on errors communicating with Azure storage.
    * @throws IOException
@@ -893,7 +910,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
         STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME,
         DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME));
   }
-  
+
   @VisibleForTesting
   public static String getAccountKeyFromConfiguration(String accountName,
       Configuration conf) throws KeyProviderException {
@@ -930,7 +947,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * Establish a session with Azure blob storage based on the target URI. The
    * method determines whether or not the URI target contains an explicit
    * account or an implicit default cluster-wide account.
-   * 
+   *
    * @throws AzureException
    * @throws IOException
    */
@@ -983,7 +1000,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       instrumentation.setAccountName(accountName);
       String containerName = getContainerFromAuthority(sessionUri);
       instrumentation.setContainerName(containerName);
-      
+
       // Check whether this is a storage emulator account.
       if (isStorageEmulatorAccount(accountName)) {
         // It is an emulator account, connect to it with no credentials.
@@ -1086,7 +1103,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    */
   private String verifyAndConvertToStandardFormat(String rawDir) throws URISyntaxException {
     URI asUri = new URI(rawDir);
-    if (asUri.getAuthority() == null 
+    if (asUri.getAuthority() == null
         || asUri.getAuthority().toLowerCase(Locale.ENGLISH).equalsIgnoreCase(
       sessionUri.getAuthority().toLowerCase(Locale.ENGLISH))) {
       // Applies to me.
@@ -1167,8 +1184,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     return false;
   }
 
-  
-  
+
+
   /**
    * This should be called from any method that does any modifications to the
    * underlying container: it makes sure to put the WASB current version in the
@@ -1364,11 +1381,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
        *      could mean either:
        *        (1) container=mycontainer; blob=myblob.txt
        *        (2) container=$root; blob=mycontainer/myblob.txt
-       * 
+       *
        * To avoid this type of ambiguity the Azure blob storage prevents
        * arbitrary path under $root. For a simple and more consistent user
        * experience it was decided to eliminate the opportunity for creating
-       * such paths by making the $root container read-only under WASB. 
+       * such paths by making the $root container read-only under WASB.
        */
 
       // Check that no attempt is made to write to blobs on default
@@ -1445,7 +1462,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Default permission to use when no permission metadata is found.
-   * 
+   *
    * @return The default permission to use.
    */
   private static PermissionStatus defaultPermissionNoBlobMetadata() {
@@ -1688,7 +1705,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   /**
    * Private method to check for authenticated access.
-   * 
+   *
    * @ returns boolean -- true if access is credentialed and authenticated and
    * false otherwise.
    */
@@ -1708,7 +1725,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * original file system object was constructed with a short- or long-form URI.
    * If the root directory is non-null the URI in the file constructor was in
    * the long form.
-   * 
+   *
    * @param includeMetadata
    *          if set, the listed items will have their metadata populated
    *          already.
@@ -1717,7 +1734,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    *
    * @returns blobItems : iterable collection of blob items.
    * @throws URISyntaxException
-   * 
+   *
    */
   private Iterable<ListBlobItem> listRootBlobs(boolean includeMetadata,
       boolean useFlatBlobListing) throws StorageException, URISyntaxException {
@@ -1736,7 +1753,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * the directory depending on whether the original file system object was
    * constructed with a short- or long-form URI. If the root directory is
    * non-null the URI in the file constructor was in the long form.
-   * 
+   *
    * @param aPrefix
    *          : string name representing the prefix of containing blobs.
    * @param includeMetadata
@@ -1744,10 +1761,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    *          already.
    * @param useFlatBlobListing
    *          if set the list is flat, otherwise it is hierarchical.
-   * 
+   *
    * @returns blobItems : iterable collection of blob items.
    * @throws URISyntaxException
-   * 
+   *
    */
   private Iterable<ListBlobItem> listRootBlobs(String aPrefix, boolean includeMetadata,
       boolean useFlatBlobListing) throws StorageException, URISyntaxException {
@@ -1769,7 +1786,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * constructed with a short- or long-form URI. It also uses the specified flat
    * or hierarchical option, listing details options, request options, and
    * operation context.
-   * 
+   *
    * @param aPrefix
    *          string name representing the prefix of containing blobs.
    * @param useFlatBlobListing
@@ -1784,7 +1801,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    *          - context of the current operation
    * @returns blobItems : iterable collection of blob items.
    * @throws URISyntaxException
-   * 
+   *
    */
   private Iterable<ListBlobItem> listRootBlobs(String aPrefix, boolean useFlatBlobListing,
       EnumSet<BlobListingDetails> listingDetails, BlobRequestOptions options,
@@ -1804,13 +1821,13 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * get the block blob reference depending on whether the original file system
    * object was constructed with a short- or long-form URI. If the root
    * directory is non-null the URI in the file constructor was in the long form.
-   * 
+   *
    * @param aKey
    *          : a key used to query Azure for the block blob.
    * @returns blob : a reference to the Azure block blob corresponding to the
    *          key.
    * @throws URISyntaxException
-   * 
+   *
    */
   private CloudBlobWrapper getBlobReference(String aKey)
       throws StorageException, URISyntaxException {
@@ -1831,10 +1848,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * This private method normalizes the key by stripping the container name from
    * the path and returns a path relative to the root directory of the
    * container.
-   * 
+   *
    * @param keyUri
    *          - adjust this key to a path relative to the root directory
-   * 
+   *
    * @returns normKey
    */
   private String normalizeKey(URI keyUri) {
@@ -1853,11 +1870,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * This private method normalizes the key by stripping the container name from
    * the path and returns a path relative to the root directory of the
    * container.
-   * 
+   *
    * @param blob
    *          - adjust the key to this blob to a path relative to the root
    *          directory
-   * 
+   *
    * @returns normKey
    */
   private String normalizeKey(CloudBlobWrapper blob) {
@@ -1868,11 +1885,11 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * This private method normalizes the key by stripping the container name from
    * the path and returns a path relative to the root directory of the
    * container.
-   * 
+   *
    * @param directory
    *          - adjust the key to this directory to a path relative to the root
    *          directory
-   * 
+   *
    * @returns normKey
    */
   private String normalizeKey(CloudBlobDirectoryWrapper directory) {
@@ -1889,7 +1906,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * operation that has listeners hooked to it that will update the metrics for
    * this file system. This method does not bind to receive send request
    * callbacks by default.
-   * 
+   *
    * @return The OperationContext object to use.
    */
   private OperationContext getInstrumentedContext() {
@@ -1900,16 +1917,27 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Creates a new OperationContext for the Azure Storage operation that has
    * listeners hooked to it that will update the metrics for this file system.
-   * 
+   *
    * @param bindConcurrentOOBIo
    *          - bind to intercept send request call backs to handle OOB I/O.
-   * 
+   *
    * @return The OperationContext object to use.
    */
   private OperationContext getInstrumentedContext(boolean bindConcurrentOOBIo) {
 
     OperationContext operationContext = new OperationContext();
 
+    // Set User-Agent
+    operationContext.getSendingRequestEventHandler().addListener(new StorageEvent<SendingRequestEvent>() {
+      @Override
+      public void eventOccurred(SendingRequestEvent eventArg) {
+        HttpURLConnection connection = (HttpURLConnection) eventArg.getConnectionObject();
+        String userAgentInfo = String.format(Utility.LOCALE_US, "WASB/%s (%s) %s",
+                VersionInfo.getVersion(), userAgentId, BaseRequest.getUserAgent());
+        connection.setRequestProperty(Constants.HeaderConstants.USER_AGENT, userAgentInfo);
+      }
+    });
+
     if (selfThrottlingEnabled) {
       SelfThrottlingIntercept.hook(operationContext, selfThrottlingReadFactor,
           selfThrottlingWriteFactor);
@@ -2096,7 +2124,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   /**
    * Searches the given list of {@link FileMetadata} objects for a directory
    * with the given key.
-   * 
+   *
    * @param list
    *          The list to search.
    * @param key
@@ -2229,7 +2257,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    * Build up a metadata list of blobs in an Azure blob directory. This method
    * uses a in-order first traversal of blob directory structures to maintain
    * the sorted order of the blob names.
-   * 
+   *
    * @param aCloudBlobDirectory Azure blob directory
    * @param aFileMetadataList a list of file metadata objects for each
    *                          non-directory blob.
@@ -2564,7 +2592,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       //
       // Copy blob operation in Azure storage is very costly. It will be highly
       // likely throttled during Azure storage gc. Short term fix will be using
-      // a more intensive exponential retry policy when the cluster is getting 
+      // a more intensive exponential retry policy when the cluster is getting
       // throttled.
       try {
         dstBlob.startCopyFromBlob(srcBlob, null, getInstrumentedContext());
@@ -2585,10 +2613,10 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           int copyBlobMaxRetries = sessionConfiguration.getInt(
             KEY_COPYBLOB_MAX_IO_RETRIES,
 			DEFAULT_COPYBLOB_MAX_RETRY_ATTEMPTS);
-	        
+
           BlobRequestOptions options = new BlobRequestOptions();
           options.setRetryPolicyFactory(new RetryExponentialRetry(
-            copyBlobMinBackoff, copyBlobDeltaBackoff, copyBlobMaxBackoff, 
+            copyBlobMinBackoff, copyBlobDeltaBackoff, copyBlobMaxBackoff,
             copyBlobMaxRetries));
           dstBlob.startCopyFromBlob(srcBlob, options, getInstrumentedContext());
         } else {
@@ -2794,7 +2822,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       bandwidthGaugeUpdater = null;
     }
   }
-  
+
   // Finalizer to ensure complete shutdown
   @Override
   protected void finalize() throws Throwable {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/hadoop-tools/hadoop-azure/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
index 7415e29..79cb0ea 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
@@ -192,6 +192,19 @@ The configuration option `fs.azure.page.blob.extension.size` is the page blob
 extension size.  This defines the amount to extend a page blob if it starts to
 get full.  It must be 128MB or greater, specified as an integer number of bytes.
 
+### Custom User-Agent
+WASB passes User-Agent header to the Azure back-end. The default value
+contains WASB version, Java Runtime version, Azure Client library version, and the
+value of the configuration option `fs.azure.user.agent.prefix`. Customized User-Agent
+header enables better troubleshooting and analysis by Azure service.
+
+```xml
+<property>
+    <name>fs.azure.user.agent.prefix</name>
+    <value>Identifier</value>
+</property>
+```
+
 ### Atomic Folder Rename
 
 Azure storage stores files as a flat key/value store without formal support

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
index 194a831..672ed9c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
@@ -566,4 +566,52 @@ public class TestWasbUriAndConfiguration {
         CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, null);
     assertEquals(newPath, effectivePath);
   }
+
+  @Test
+  public void testUserAgentConfig() throws Exception {
+    // Set the user agent
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
+
+      conf.set(AzureNativeFileSystemStore.USER_AGENT_ID_KEY, "TestClient");
+
+      FileSystem fs = FileSystem.get(conf);
+      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
+
+      assertTrue(afs instanceof Wasbs);
+      assertEquals(-1, afs.getUri().getPort());
+      assertEquals("wasbs", afs.getUri().getScheme());
+
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+
+    // Unset the user agent
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
+
+      conf.unset(AzureNativeFileSystemStore.USER_AGENT_ID_KEY);
+
+      FileSystem fs = FileSystem.get(conf);
+      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
+      assertTrue(afs instanceof Wasbs);
+      assertEquals(-1, afs.getUri().getPort());
+      assertEquals("wasbs", afs.getUri().getScheme());
+
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2921e51/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
index acd9459..8c88743 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
@@ -34,6 +34,11 @@
     <value>true</value>
   </property>
 
+  <property>
+    <name>fs.azure.user.agent.prefix</name>
+    <value>MSFT</value>
+  </property>
+
   <!-- Save the above configuration properties in a separate file named -->
   <!-- azure-auth-keys.xml in the same directory as this file. -->
   <!-- DO NOT ADD azure-auth-keys.xml TO REVISION CONTROL.  The keys to your -->


---------------------------------------------------------------------
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: HDFS-12166. Do not deprecate HTTPFS_TEMP. Contributed by John Zhuge.

Posted by in...@apache.org.
HDFS-12166. Do not deprecate HTTPFS_TEMP. Contributed by John Zhuge.

Change-Id: Icdd032de9feea3d0dde71bf8145b70497993d97f


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

Branch: refs/heads/HDFS-10467
Commit: 3b48f81411230d1cfa7b0f6ad82f44cb0dc0efa6
Parents: 44350fd
Author: John Zhuge <jz...@cloudera.com>
Authored: Wed Jul 19 17:37:33 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Thu Jul 20 20:45:09 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh  |  2 +-
 .../fs/http/server/HttpFSServerWebServer.java       |  2 --
 .../main/libexec/shellprofile.d/hadoop-httpfs.sh    | 16 ++++++++++------
 .../src/site/markdown/ServerSetup.md.vm             |  1 -
 4 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b48f814/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
index 6c0a80f..5b7b05d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
@@ -28,7 +28,7 @@
 
 # HTTPFS temporary directory
 #
-# export HTTPFS_TEMP=${HADOOP_HOME}/temp
+# export HTTPFS_TEMP=${HADOOP_HDFS_HOME}/temp
 
 # The HTTP port used by HTTPFS
 #

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b48f814/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebServer.java
index 0949f86..39f1caf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebServer.java
@@ -71,8 +71,6 @@ public class HttpFSServerWebServer {
   HttpFSServerWebServer(Configuration conf, Configuration sslConf) throws
       Exception {
     // Override configuration with deprecated environment variables.
-    deprecateEnv("HTTPFS_TEMP", conf, HttpServer2.HTTP_TEMP_DIR_KEY,
-        HTTPFS_SITE_XML);
     deprecateEnv("HTTPFS_HTTP_HOSTNAME", conf, HTTP_HOSTNAME_KEY,
         HTTPFS_SITE_XML);
     deprecateEnv("HTTPFS_HTTP_PORT", conf, HTTP_PORT_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b48f814/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/shellprofile.d/hadoop-httpfs.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/shellprofile.d/hadoop-httpfs.sh b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/shellprofile.d/hadoop-httpfs.sh
old mode 100644
new mode 100755
index 239409c..6301e27
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/shellprofile.d/hadoop-httpfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/shellprofile.d/hadoop-httpfs.sh
@@ -37,14 +37,18 @@ function hdfs_subcommand_httpfs
   # shellcheck disable=SC2034
 
   hadoop_add_param HADOOP_OPTS "-Dhttpfs.home.dir" \
-    "-Dhttpfs.home.dir=${HADOOP_HOME}"
+    "-Dhttpfs.home.dir=${HTTPFS_HOME:-${HADOOP_HDFS_HOME}}"
   hadoop_add_param HADOOP_OPTS "-Dhttpfs.config.dir" \
     "-Dhttpfs.config.dir=${HTTPFS_CONFIG:-${HADOOP_CONF_DIR}}"
   hadoop_add_param HADOOP_OPTS "-Dhttpfs.log.dir" \
     "-Dhttpfs.log.dir=${HTTPFS_LOG:-${HADOOP_LOG_DIR}}"
 
-  if [[ "${HADOOP_DAEMON_MODE}" == "default" ]] ||
-     [[ "${HADOOP_DAEMON_MODE}" == "start" ]]; then
-    hadoop_mkdir "${HTTPFS_TEMP:-${HADOOP_HOME}/temp}"
-  fi
-}
\ No newline at end of file
+  local temp_dir=${HTTPFS_TEMP:-${HADOOP_HDFS_HOME}/temp}
+  hadoop_add_param HADOOP_OPTS "-Dhttpfs.temp.dir" \
+    "-Dhttpfs.temp.dir=${temp_dir}"
+  case ${HADOOP_DAEMON_MODE} in
+    start|default)
+      hadoop_mkdir "${temp_dir}"
+    ;;
+  esac
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3b48f814/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
index 91ef90e..072c067 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/markdown/ServerSetup.md.vm
@@ -149,7 +149,6 @@ HTTPFS_MAX_THREADS          | hadoop.http.max.threads      | httpfs-site.xml
 HTTPFS_SSL_ENABLED          | httpfs.ssl.enabled           | httpfs-site.xml
 HTTPFS_SSL_KEYSTORE_FILE    | ssl.server.keystore.location | ssl-server.xml
 HTTPFS_SSL_KEYSTORE_PASS    | ssl.server.keystore.password | ssl-server.xml
-HTTPFS_TEMP                 | hadoop.http.temp.dir         | httpfs-site.xml
 
 HTTP Default Services
 ---------------------


---------------------------------------------------------------------
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: HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: deb0a67103c1c4da0112e0e2c88fcc8b020b0b40
Parents: 369f731
Author: Inigo <in...@apache.org>
Authored: Tue Mar 28 14:30:59 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 28 09:48:38 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/hdfs               |   5 +
 .../hadoop-hdfs/src/main/bin/hdfs.cmd           |   8 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  17 +
 .../resolver/ActiveNamenodeResolver.java        | 117 +++
 .../resolver/FederationNamenodeContext.java     |  87 +++
 .../FederationNamenodeServiceState.java         |  46 ++
 .../resolver/FederationNamespaceInfo.java       |  99 +++
 .../resolver/FileSubclusterResolver.java        |  75 ++
 .../resolver/NamenodePriorityComparator.java    |  63 ++
 .../resolver/NamenodeStatusReport.java          | 195 +++++
 .../federation/resolver/PathLocation.java       | 122 +++
 .../federation/resolver/RemoteLocation.java     |  74 ++
 .../federation/resolver/package-info.java       |  41 +
 .../federation/router/FederationUtil.java       | 117 +++
 .../router/RemoteLocationContext.java           |  38 +
 .../hdfs/server/federation/router/Router.java   | 263 +++++++
 .../federation/router/RouterRpcServer.java      | 102 +++
 .../server/federation/router/package-info.java  |  31 +
 .../federation/store/StateStoreService.java     |  77 ++
 .../server/federation/store/package-info.java   |  62 ++
 .../src/main/resources/hdfs-default.xml         |  16 +
 .../server/federation/FederationTestUtils.java  | 233 ++++++
 .../hdfs/server/federation/MockResolver.java    | 290 +++++++
 .../server/federation/RouterConfigBuilder.java  |  40 +
 .../server/federation/RouterDFSCluster.java     | 767 +++++++++++++++++++
 .../server/federation/router/TestRouter.java    |  96 +++
 26 files changed, 3080 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 594a468..6606979 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -57,6 +57,7 @@ function hadoop_usage
   hadoop_add_subcommand "oiv" "apply the offline fsimage viewer to an fsimage"
   hadoop_add_subcommand "oiv_legacy" "apply the offline fsimage viewer to a legacy fsimage"
   hadoop_add_subcommand "portmap" "run a portmap service"
+  hadoop_add_subcommand "router" "run the DFS router"
   hadoop_add_subcommand "secondarynamenode" "run the DFS secondary namenode"
   hadoop_add_subcommand "snapshotDiff" "diff two snapshots of a directory or diff the current directory contents with a snapshot"
   hadoop_add_subcommand "storagepolicies" "list/get/set block storage policies"
@@ -176,6 +177,10 @@ function hdfscmd_case
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME=org.apache.hadoop.portmap.Portmap
     ;;
+    router)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.federation.router.Router'
+    ;;
     secondarynamenode)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
index 2181e47..b9853d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
@@ -59,7 +59,7 @@ if "%1" == "--loglevel" (
     )
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto debug
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath crypto router debug
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -179,6 +179,11 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.CryptoAdmin
   goto :eof
 
+:router
+  set CLASS=org.apache.hadoop.hdfs.server.federation.router.Router
+  set HADOOP_OPTS=%HADOOP_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
 :debug
   set CLASS=org.apache.hadoop.hdfs.tools.DebugAdmin
   goto :eof
@@ -219,6 +224,7 @@ goto :eof
   @echo   secondarynamenode    run the DFS secondary namenode
   @echo   namenode             run the DFS namenode
   @echo   journalnode          run the DFS journalnode
+  @echo   router               run the DFS router
   @echo   zkfc                 run the ZK Failover Controller daemon
   @echo   datanode             run a DFS datanode
   @echo   dfsadmin             run a DFS admin client

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 1f60f32..71f999a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -1087,6 +1087,23 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.use.dfs.network.topology";
   public static final boolean DFS_USE_DFS_NETWORK_TOPOLOGY_DEFAULT = true;
 
+  // HDFS federation
+  public static final String FEDERATION_PREFIX = "dfs.federation.";
+
+  // HDFS Router-based federation
+  public static final String FEDERATION_ROUTER_PREFIX =
+      "dfs.federation.router.";
+
+  // HDFS Router State Store connection
+  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "file.resolver.client.class";
+  public static final String FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT =
+      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS =
+      FEDERATION_ROUTER_PREFIX + "namenode.resolver.client.class";
+  public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
+      "org.apache.hadoop.hdfs.server.federation.MockResolver";
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.java
new file mode 100644
index 0000000..477053d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/ActiveNamenodeResolver.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.hdfs.server.federation.resolver;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Locates the most active NN for a given nameservice ID or blockpool ID. This
+ * interface is used by the {@link org.apache.hadoop.hdfs.server.federation.
+ * router.RouterRpcServer RouterRpcServer} to:
+ * <ul>
+ * <li>Determine the target NN for a given subcluster.
+ * <li>List of all namespaces discovered/active in the federation.
+ * <li>Update the currently active NN empirically.
+ * </ul>
+ * The interface is also used by the {@link org.apache.hadoop.hdfs.server.
+ * federation.router.NamenodeHeartbeatService NamenodeHeartbeatService} to
+ * register a discovered NN.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ActiveNamenodeResolver {
+
+  /**
+   * Report a successful, active NN address for a nameservice or blockPool.
+   *
+   * @param ns Nameservice identifier.
+   * @param successfulAddress The address the successful responded to the
+   *                          command.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  void updateActiveNamenode(
+      String ns, InetSocketAddress successfulAddress) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single nameservice ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param nameserviceId Nameservice identifier.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForNameserviceId(String nameserviceId) throws IOException;
+
+  /**
+   * Returns a prioritized list of the most recent cached registration entries
+   * for a single block pool ID.
+   * Returns an empty list if none are found. Returns entries in preference of:
+   * <ul>
+   * <li>The most recent ACTIVE NN
+   * <li>The most recent STANDBY NN
+   * <li>The most recent UNAVAILABLE NN
+   * </ul>
+   *
+   * @param blockPoolId Block pool identifier for the nameservice.
+   * @return Prioritized list of namenode contexts.
+   * @throws IOException If the state store cannot be accessed.
+   */
+  List<? extends FederationNamenodeContext>
+      getNamenodesForBlockPoolId(String blockPoolId) throws IOException;
+
+  /**
+   * Register a namenode in the State Store.
+   *
+   * @param report Namenode status report.
+   * @return True if the node was registered and successfully committed to the
+   *         data store.
+   * @throws IOException Throws exception if the namenode could not be
+   *           registered.
+   */
+  boolean registerNamenode(NamenodeStatusReport report) throws IOException;
+
+  /**
+   * Get a list of all namespaces that are registered and active in the
+   * federation.
+   *
+   * @return List of name spaces in the federation
+   * @throws Throws exception if the namespace list is not available.
+   */
+  Set<FederationNamespaceInfo> getNamespaces() throws IOException;
+
+  /**
+   * Assign a unique identifier for the parent router service.
+   * Required to report the status to the namenode resolver.
+   *
+   * @param router Unique string identifier for the router.
+   */
+  void setRouterId(String routerId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
new file mode 100644
index 0000000..68ef02a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeContext.java
@@ -0,0 +1,87 @@
+/**
+ * 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.resolver;
+
+/**
+ * Interface for a discovered NN and its current server endpoints.
+ */
+public interface FederationNamenodeContext {
+
+  /**
+   * Get the RPC server address of the namenode.
+   *
+   * @return RPC server address in the form of host:port.
+   */
+  String getRpcAddress();
+
+  /**
+   * Get the Service RPC server address of the namenode.
+   *
+   * @return Service RPC server address in the form of host:port.
+   */
+  String getServiceAddress();
+
+  /**
+   * Get the Lifeline RPC server address of the namenode.
+   *
+   * @return Lifeline RPC server address in the form of host:port.
+   */
+  String getLifelineAddress();
+
+  /**
+   * Get the HTTP server address of the namenode.
+   *
+   * @return HTTP address in the form of host:port.
+   */
+  String getWebAddress();
+
+  /**
+   * Get the unique key representing the namenode.
+   *
+   * @return Combination of the nameservice and the namenode IDs.
+   */
+  String getNamenodeKey();
+
+  /**
+   * Identifier for the nameservice/namespace.
+   *
+   * @return Namenode nameservice identifier.
+   */
+  String getNameserviceId();
+
+  /**
+   * Identifier for the namenode.
+   *
+   * @return String
+   */
+  String getNamenodeId();
+
+  /**
+   * The current state of the namenode (active, standby, etc).
+   *
+   * @return FederationNamenodeServiceState State of the namenode.
+   */
+  FederationNamenodeServiceState getState();
+
+  /**
+   * The update date.
+   *
+   * @return Long with the update date.
+   */
+  long getDateModified();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
new file mode 100644
index 0000000..c773f82
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamenodeServiceState.java
@@ -0,0 +1,46 @@
+/**
+ * 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.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+
+/**
+ * Namenode state in the federation. The order of this enum is used to evaluate
+ * NN priority for RPC calls.
+ */
+public enum FederationNamenodeServiceState {
+  ACTIVE, // HAServiceState.ACTIVE or operational.
+  STANDBY, // HAServiceState.STANDBY.
+  UNAVAILABLE, // When the namenode cannot be reached.
+  EXPIRED; // When the last update is too old.
+
+  public static FederationNamenodeServiceState getState(HAServiceState state) {
+    switch(state) {
+    case ACTIVE:
+      return FederationNamenodeServiceState.ACTIVE;
+    case STANDBY:
+      return FederationNamenodeServiceState.STANDBY;
+    case INITIALIZING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    case STOPPING:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    default:
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
new file mode 100644
index 0000000..bbaeca3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FederationNamespaceInfo.java
@@ -0,0 +1,99 @@
+/**
+ * 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.resolver;
+
+import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
+
+/**
+ * Represents information about a single nameservice/namespace in a federated
+ * HDFS cluster.
+ */
+public class FederationNamespaceInfo
+    implements Comparable<FederationNamespaceInfo>, RemoteLocationContext {
+
+  /** Block pool identifier. */
+  private String blockPoolId;
+  /** Cluster identifier. */
+  private String clusterId;
+  /** Nameservice identifier. */
+  private String nameserviceId;
+
+  public FederationNamespaceInfo(String bpId, String clId, String nsId) {
+    this.blockPoolId = bpId;
+    this.clusterId = clId;
+    this.nameserviceId = nsId;
+  }
+
+  /**
+   * The HDFS nameservice id for this namespace.
+   *
+   * @return Nameservice identifier.
+   */
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * The HDFS cluster id for this namespace.
+   *
+   * @return Cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * The HDFS block pool id for this namespace.
+   *
+   * @return Block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.nameserviceId.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    } else if (obj instanceof FederationNamespaceInfo) {
+      return this.compareTo((FederationNamespaceInfo) obj) == 0;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int compareTo(FederationNamespaceInfo info) {
+    return this.nameserviceId.compareTo(info.getNameserviceId());
+  }
+
+  @Override
+  public String toString() {
+    return this.nameserviceId + "->" + this.blockPoolId + ":" + this.clusterId;
+  }
+
+  @Override
+  public String getDest() {
+    return this.nameserviceId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
new file mode 100644
index 0000000..af9f493
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/FileSubclusterResolver.java
@@ -0,0 +1,75 @@
+/**
+ * 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.resolver;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Interface to map a file path in the global name space to a specific
+ * subcluster and path in an HDFS name space.
+ * <p>
+ * Each path in the global/federated namespace may map to 1-N different HDFS
+ * locations.  Each location specifies a single nameservice and a single HDFS
+ * path.  The behavior is similar to MergeFS and Nfly and allows the merger
+ * of multiple HDFS locations into a single path.  See HADOOP-8298 and
+ * HADOOP-12077
+ * <p>
+ * For example, a directory listing will fetch listings for each destination
+ * path and combine them into a single set of results.
+ * <p>
+ * When multiple destinations are available for a path, the destinations are
+ * prioritized in a consistent manner.  This allows the proxy server to
+ * guess the best/most likely destination and attempt it first.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface FileSubclusterResolver {
+
+  /**
+   * Get the destinations for a global path. Results are from the mount table
+   * cache.  If multiple destinations are available, the first result is the
+   * highest priority destination.
+   *
+   * @param path Global path.
+   * @return Location in a destination namespace or null if it does not exist.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  PathLocation getDestinationForPath(String path) throws IOException;
+
+  /**
+   * Get a list of mount points for a path. Results are from the mount table
+   * cache.
+   *
+   * @return List of mount points present at this path or zero-length list if
+   *         none are found.
+   * @throws IOException Throws exception if the data is not available.
+   */
+  List<String> getMountPoints(String path) throws IOException;
+
+  /**
+   * Get the default namespace for the cluster.
+   *
+   * @return Default namespace identifier.
+   */
+  String getDefaultNamespace();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
new file mode 100644
index 0000000..fe82f29
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodePriorityComparator.java
@@ -0,0 +1,63 @@
+/**
+ * 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.resolver;
+
+import java.util.Comparator;
+
+/**
+ * Compares NNs in the same namespace and prioritizes by their status. The
+ * priorities are:
+ * <ul>
+ * <li>ACTIVE
+ * <li>STANDBY
+ * <li>UNAVAILABLE
+ * </ul>
+ * When two NNs have the same state, the last modification date is the tie
+ * breaker, newest has priority. Expired NNs are excluded.
+ */
+public class NamenodePriorityComparator
+    implements Comparator<FederationNamenodeContext> {
+
+  @Override
+  public int compare(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    FederationNamenodeServiceState state1 = o1.getState();
+    FederationNamenodeServiceState state2 = o2.getState();
+
+    if (state1 == state2) {
+      // Both have the same state, use mode dates
+      return compareModDates(o1, o2);
+    } else {
+      // Enum is ordered by priority
+      return state1.compareTo(state2);
+    }
+  }
+
+  /**
+   * Compare the modification dates.
+   *
+   * @param o1 Context 1.
+   * @param o2 Context 2.
+   * @return Comparison between dates.
+   */
+  private int compareModDates(FederationNamenodeContext o1,
+      FederationNamenodeContext o2) {
+    // Reverse sort, lowest position is highest priority.
+    return (int) (o2.getDateModified() - o1.getDateModified());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
new file mode 100644
index 0000000..9259048
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+/**
+ * Status of the namenode.
+ */
+public class NamenodeStatusReport {
+
+  /** Namenode information. */
+  private String nameserviceId = "";
+  private String namenodeId = "";
+  private String clusterId = "";
+  private String blockPoolId = "";
+  private String rpcAddress = "";
+  private String serviceAddress = "";
+  private String lifelineAddress = "";
+  private String webAddress = "";
+
+  /** Namenode state. */
+  private HAServiceState status = HAServiceState.STANDBY;
+  private boolean safeMode = false;
+
+  /** If the fields are valid. */
+  private boolean registrationValid = false;
+  private boolean haStateValid = false;
+
+  public NamenodeStatusReport(String ns, String nn, String rpc, String service,
+      String lifeline, String web) {
+    this.nameserviceId = ns;
+    this.namenodeId = nn;
+    this.rpcAddress = rpc;
+    this.serviceAddress = service;
+    this.lifelineAddress = lifeline;
+    this.webAddress = web;
+  }
+
+  /**
+   * If the registration is valid.
+   *
+   * @return If the registration is valid.
+   */
+  public boolean registrationValid() {
+    return this.registrationValid;
+  }
+
+  /**
+   * If the HA state is valid.
+   *
+   * @return If the HA state is valid.
+   */
+  public boolean haStateValid() {
+    return this.haStateValid;
+  }
+
+  /**
+   * Get the state of the Namenode being monitored.
+   *
+   * @return State of the Namenode.
+   */
+  public FederationNamenodeServiceState getState() {
+    if (!registrationValid) {
+      return FederationNamenodeServiceState.UNAVAILABLE;
+    } else if (haStateValid) {
+      return FederationNamenodeServiceState.getState(status);
+    } else {
+      return FederationNamenodeServiceState.ACTIVE;
+    }
+  }
+
+  /**
+   * Get the name service identifier.
+   *
+   * @return The name service identifier.
+   */
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  /**
+   * Get the namenode identifier.
+   *
+   * @return The namenode identifier.
+   */
+  public String getNamenodeId() {
+    return this.namenodeId;
+  }
+
+  /**
+   * Get the cluster identifier.
+   *
+   * @return The cluster identifier.
+   */
+  public String getClusterId() {
+    return this.clusterId;
+  }
+
+  /**
+   * Get the block pool identifier.
+   *
+   * @return The block pool identifier.
+   */
+  public String getBlockPoolId() {
+    return this.blockPoolId;
+  }
+
+  /**
+   * Get the RPC address.
+   *
+   * @return The RPC address.
+   */
+  public String getRpcAddress() {
+    return this.rpcAddress;
+  }
+
+  /**
+   * Get the Service RPC address.
+   *
+   * @return The Service RPC address.
+   */
+  public String getServiceAddress() {
+    return this.serviceAddress;
+  }
+
+  /**
+   * Get the Lifeline RPC address.
+   *
+   * @return The Lifeline RPC address.
+   */
+  public String getLifelineAddress() {
+    return this.lifelineAddress;
+  }
+
+  /**
+   * Get the web address.
+   *
+   * @return The web address.
+   */
+  public String getWebAddress() {
+    return this.webAddress;
+  }
+
+  /**
+   * Get the HA service state.
+   *
+   * @return The HA service state.
+   */
+  public void setHAServiceState(HAServiceState state) {
+    this.status = state;
+    this.haStateValid = true;
+  }
+
+  /**
+   * Set the namespace information.
+   *
+   * @param info Namespace information.
+   */
+  public void setNamespaceInfo(NamespaceInfo info) {
+    this.clusterId = info.getClusterID();
+    this.blockPoolId = info.getBlockPoolID();
+    this.registrationValid = true;
+  }
+
+  public void setSafeMode(boolean safemode) {
+    this.safeMode = safemode;
+  }
+
+  public boolean getSafemode() {
+    return this.safeMode;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s-%s:%s",
+        nameserviceId, namenodeId, serviceAddress);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
new file mode 100644
index 0000000..d90565c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
@@ -0,0 +1,122 @@
+/**
+ * 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.resolver;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A map of the properties and target destinations (name space + path) for
+ * a path in the global/federated namespace.
+ * This data is generated from the @see MountTable records.
+ */
+public class PathLocation {
+
+  /** Source path in global namespace. */
+  private final String sourcePath;
+
+  /** Remote paths in the target namespaces. */
+  private final List<RemoteLocation> destinations;
+
+  /** List of name spaces present. */
+  private final Set<String> namespaces;
+
+
+  /**
+   * Create a new PathLocation.
+   *
+   * @param source Source path in the global name space.
+   * @param dest Destinations of the mount table entry.
+   * @param namespaces Unique identifier representing the combination of
+   *          name spaces present in the destination list.
+   */
+  public PathLocation(
+      String source, List<RemoteLocation> dest, Set<String> nss) {
+    this.sourcePath = source;
+    this.destinations = dest;
+    this.namespaces = nss;
+  }
+
+  /**
+   * Create a path location from another path.
+   *
+   * @param other Other path location to copy from.
+   */
+  public PathLocation(PathLocation other) {
+    this.sourcePath = other.sourcePath;
+    this.destinations = new LinkedList<RemoteLocation>(other.destinations);
+    this.namespaces = new HashSet<String>(other.namespaces);
+  }
+
+  /**
+   * Get the source path in the global namespace for this path location.
+   *
+   * @return The path in the global namespace.
+   */
+  public String getSourcePath() {
+    return this.sourcePath;
+  }
+
+  /**
+   * Get the list of subclusters defined for the destinations.
+   */
+  public Set<String> getNamespaces() {
+    return Collections.unmodifiableSet(this.namespaces);
+  }
+
+  @Override
+  public String toString() {
+    RemoteLocation loc = getDefaultLocation();
+    return loc.getNameserviceId() + "->" + loc.getDest();
+  }
+
+  /**
+   * Check if this location supports multiple clusters/paths.
+   *
+   * @return If it has multiple destinations.
+   */
+  public boolean hasMultipleDestinations() {
+    return this.destinations.size() > 1;
+  }
+
+  /**
+   * Get the list of locations found in the mount table.
+   * The first result is the highest priority path.
+   *
+   * @return List of remote locations.
+   */
+  public List<RemoteLocation> getDestinations() {
+    return Collections.unmodifiableList(this.destinations);
+  }
+
+  /**
+   * Get the default or highest priority location.
+   *
+   * @return The default location.
+   */
+  public RemoteLocation getDefaultLocation() {
+    if (destinations.isEmpty() || destinations.get(0).getDest() == null) {
+      throw new UnsupportedOperationException(
+          "Unsupported path " + sourcePath + " please check mount table");
+    }
+    return destinations.get(0);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
new file mode 100644
index 0000000..eef136d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RemoteLocation.java
@@ -0,0 +1,74 @@
+/**
+ * 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.resolver;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext;
+
+/**
+ * A single in a remote namespace consisting of a nameservice ID
+ * and a HDFS path.
+ */
+public class RemoteLocation implements RemoteLocationContext {
+
+  /** Identifier of the remote namespace for this location. */
+  private String nameserviceId;
+  /** Path in the remote location. */
+  private String path;
+
+  /**
+   * Create a new remote location.
+   *
+   * @param nsId Destination namespace.
+   * @param pPath Path in the destination namespace.
+   */
+  public RemoteLocation(String nsId, String pPath) {
+    this.nameserviceId = nsId;
+    this.path = pPath;
+  }
+
+  @Override
+  public String getNameserviceId() {
+    return this.nameserviceId;
+  }
+
+  @Override
+  public String getDest() {
+    return this.path;
+  }
+
+  @Override
+  public String toString() {
+    return this.nameserviceId + "->" + this.path;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 31)
+        .append(this.nameserviceId)
+        .append(this.path)
+        .toHashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return (obj != null &&
+        obj.getClass() == this.getClass() &&
+        obj.hashCode() == this.hashCode());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.java
new file mode 100644
index 0000000..d8be9e3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/package-info.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.
+ */
+
+/**
+ * The resolver package contains indepedent data resolvers used in HDFS
+ * federation. The data resolvers collect data from the cluster, including from
+ * the state store. The resolvers expose APIs used by HDFS federation to collect
+ * aggregated, cached data for use in Real-time request processing. The
+ * resolvers are perf-sensitive and are used in the flow of the
+ * {@link RouterRpcServer} request path.
+ * <p>
+ * The principal resolvers are:
+ * <ul>
+ * <li>{@link ActiveNamenodeResolver} Real-time interface for locating the most
+ * recently active NN for a nameservice.
+ * <li>{@link FileSubclusterResolver} Real-time interface for determining the NN
+ * and local file path for a given file/folder based on the global namespace
+ * path.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.resolver;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
new file mode 100644
index 0000000..6e7e865
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.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.hdfs.server.federation.router;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+
+/**
+ * Utilities for managing HDFS federation.
+ */
+public final class FederationUtil {
+
+  private static final Log LOG = LogFactory.getLog(FederationUtil.class);
+
+  private FederationUtil() {
+    // Utility Class
+  }
+
+  /**
+   * Create an instance of an interface with a constructor using a state store
+   * constructor.
+   *
+   * @param conf Configuration
+   * @param context Context object to pass to the instance.
+   * @param contextType Type of the context passed to the constructor.
+   * @param configurationKeyName Configuration key to retrieve the class to load
+   * @param defaultClassName Default class to load if the configuration key is
+   *          not set
+   * @param clazz Class/interface that must be implemented by the instance.
+   * @return New instance of the specified class that implements the desired
+   *         interface and a single parameter constructor containing a
+   *         StateStore reference.
+   */
+  private static <T, R> T newInstance(final Configuration conf,
+      final R context, final Class<R> contextClass,
+      final String configKeyName, final String defaultClassName,
+      final Class<T> clazz) {
+
+    String className = conf.get(configKeyName, defaultClassName);
+    try {
+      Class<?> instance = conf.getClassByName(className);
+      if (clazz.isAssignableFrom(instance)) {
+        if (contextClass == null) {
+          // Default constructor if no context
+          @SuppressWarnings("unchecked")
+          Constructor<T> constructor =
+              (Constructor<T>) instance.getConstructor();
+          return constructor.newInstance();
+        } else {
+          // Constructor with context
+          @SuppressWarnings("unchecked")
+          Constructor<T> constructor = (Constructor<T>) instance.getConstructor(
+              Configuration.class, contextClass);
+          return constructor.newInstance(conf, context);
+        }
+      } else {
+        throw new RuntimeException("Class " + className + " not instance of "
+            + clazz.getCanonicalName());
+      }
+    } catch (ReflectiveOperationException e) {
+      LOG.error("Could not instantiate: " + className, e);
+      return null;
+    }
+  }
+
+  /**
+   * Creates an instance of a FileSubclusterResolver from the configuration.
+   *
+   * @param conf Configuration that defines the file resolver class.
+   * @param obj Context object passed to class constructor.
+   * @return FileSubclusterResolver
+   */
+  public static FileSubclusterResolver newFileSubclusterResolver(
+      Configuration conf, StateStoreService stateStore) {
+    return newInstance(conf, stateStore, StateStoreService.class,
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        FileSubclusterResolver.class);
+  }
+
+  /**
+   * Creates an instance of an ActiveNamenodeResolver from the configuration.
+   *
+   * @param conf Configuration that defines the namenode resolver class.
+   * @param obj Context object passed to class constructor.
+   * @return ActiveNamenodeResolver
+   */
+  public static ActiveNamenodeResolver newActiveNamenodeResolver(
+      Configuration conf, StateStoreService stateStore) {
+    return newInstance(conf, stateStore, StateStoreService.class,
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT,
+        ActiveNamenodeResolver.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
new file mode 100644
index 0000000..da6066b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteLocationContext.java
@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+/**
+ * Interface for objects that are unique to a namespace.
+ */
+public interface RemoteLocationContext {
+
+  /**
+   * Returns an identifier for a unique namespace.
+   *
+   * @return Namespace identifier.
+   */
+  String getNameserviceId();
+
+  /**
+   * Destination in this location. For example the path in a remote namespace.
+   *
+   * @return Destination in this location.
+   */
+  String getDest();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
new file mode 100644
index 0000000..fe0d02a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
@@ -0,0 +1,263 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newActiveNamenodeResolver;
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.newFileSubclusterResolver;
+import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Router that provides a unified view of multiple federated HDFS clusters. It
+ * has two main roles: (1) federated interface and (2) NameNode heartbeat.
+ * <p>
+ * For the federated interface, the Router receives a client request, checks the
+ * State Store for the correct subcluster, and forwards the request to the
+ * active Namenode of that subcluster. The reply from the Namenode then flows in
+ * the opposite direction. The Routers are stateless and can be behind a load
+ * balancer. HDFS clients connect to the router using the same interfaces as are
+ * used to communicate with a namenode, namely the ClientProtocol RPC interface
+ * and the WebHdfs HTTP interface exposed by the router. {@link RouterRpcServer}
+ * {@link RouterHttpServer}
+ * <p>
+ * For NameNode heartbeat, the Router periodically checks the state of a
+ * NameNode (usually on the same server) and reports their high availability
+ * (HA) state and load/space status to the State Store. Note that this is an
+ * optional role as a Router can be independent of any subcluster.
+ * {@link StateStoreService} {@link NamenodeHeartbeatService}
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class Router extends CompositeService {
+
+  private static final Log LOG = LogFactory.getLog(Router.class);
+
+
+  /** Configuration for the Router. */
+  private Configuration conf;
+
+  /** Router address/identifier. */
+  private String routerId;
+
+  /** RPC interface to the client. */
+  private RouterRpcServer rpcServer;
+
+  /** Interface with the State Store. */
+  private StateStoreService stateStore;
+
+  /** Interface to map global name space to HDFS subcluster name spaces. */
+  private FileSubclusterResolver subclusterResolver;
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private ActiveNamenodeResolver namenodeResolver;
+
+
+  /** Usage string for help message. */
+  private static final String USAGE = "Usage: java Router";
+
+  /** Priority of the Router shutdown hook. */
+  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+
+
+  /////////////////////////////////////////////////////////
+  // Constructor
+  /////////////////////////////////////////////////////////
+
+  public Router() {
+    super(Router.class.getName());
+  }
+
+  /////////////////////////////////////////////////////////
+  // Service management
+  /////////////////////////////////////////////////////////
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
+
+    // TODO Interface to the State Store
+    this.stateStore = null;
+
+    // Resolver to track active NNs
+    this.namenodeResolver = newActiveNamenodeResolver(
+        this.conf, this.stateStore);
+    if (this.namenodeResolver == null) {
+      throw new IOException("Cannot find namenode resolver.");
+    }
+
+    // Lookup interface to map between the global and subcluster name spaces
+    this.subclusterResolver = newFileSubclusterResolver(
+        this.conf, this.stateStore);
+    if (this.subclusterResolver == null) {
+      throw new IOException("Cannot find subcluster resolver");
+    }
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+
+    super.serviceStop();
+  }
+
+  /**
+   * Shutdown the router.
+   */
+  public void shutDown() {
+    new Thread() {
+      @Override
+      public void run() {
+        Router.this.stop();
+      }
+    }.start();
+  }
+
+  /**
+   * Main run loop for the router.
+   *
+   * @param argv parameters.
+   */
+  public static void main(String[] argv) {
+    if (DFSUtil.parseHelpArgument(argv, Router.USAGE, System.out, true)) {
+      System.exit(0);
+    }
+
+    try {
+      StringUtils.startupShutdownMessage(Router.class, argv, LOG);
+
+      Router router = new Router();
+
+      ShutdownHookManager.get().addShutdownHook(
+          new CompositeServiceShutdownHook(router), SHUTDOWN_HOOK_PRIORITY);
+
+      Configuration conf = new HdfsConfiguration();
+      router.init(conf);
+      router.start();
+    } catch (Throwable e) {
+      LOG.error("Failed to start router.", e);
+      terminate(1, e);
+    }
+  }
+
+  /////////////////////////////////////////////////////////
+  // RPC Server
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Create a new Router RPC server to proxy ClientProtocol requests.
+   *
+   * @return RouterRpcServer
+   * @throws IOException If the router RPC server was not started.
+   */
+  protected RouterRpcServer createRpcServer() throws IOException {
+    return new RouterRpcServer(this.conf, this, this.getNamenodeResolver(),
+        this.getSubclusterResolver());
+  }
+
+  /**
+   * Get the Router RPC server.
+   *
+   * @return Router RPC server.
+   */
+  public RouterRpcServer getRpcServer() {
+    return this.rpcServer;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Submodule getters
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Get the State Store service.
+   *
+   * @return State Store service.
+   */
+  public StateStoreService getStateStore() {
+    return this.stateStore;
+  }
+
+  /**
+   * Get the subcluster resolver for files.
+   *
+   * @return Subcluster resolver for files.
+   */
+  public FileSubclusterResolver getSubclusterResolver() {
+    return this.subclusterResolver;
+  }
+
+  /**
+   * Get the namenode resolver for a subcluster.
+   *
+   * @return The namenode resolver for a subcluster.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
+  /////////////////////////////////////////////////////////
+  // Router info
+  /////////////////////////////////////////////////////////
+
+  /**
+   * Unique ID for the router, typically the hostname:port string for the
+   * router's RPC server. This ID may be null on router startup before the RPC
+   * server has bound to a port.
+   *
+   * @return Router identifier.
+   */
+  public String getRouterId() {
+    return this.routerId;
+  }
+
+  /**
+   * Sets a unique ID for this router.
+   *
+   * @param router Identifier of the Router.
+   */
+  public void setRouterId(String id) {
+    this.routerId = id;
+    if (this.stateStore != null) {
+      this.stateStore.setIdentifier(this.routerId);
+    }
+    if (this.namenodeResolver != null) {
+      this.namenodeResolver.setRouterId(this.routerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
new file mode 100644
index 0000000..24792bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -0,0 +1,102 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.service.AbstractService;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class is responsible for handling all of the RPC calls to the It is
+ * created, started, and stopped by {@link Router}. It implements the
+ * {@link ClientProtocol} to mimic a
+ * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode} and proxies
+ * the requests to the active
+ * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode}.
+ */
+public class RouterRpcServer extends AbstractService {
+
+  /** The RPC server that listens to requests from clients. */
+  private final Server rpcServer;
+
+  /**
+   * Construct a router RPC server.
+   *
+   * @param configuration HDFS Configuration.
+   * @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.
+   */
+  public RouterRpcServer(Configuration configuration, Router router,
+      ActiveNamenodeResolver nnResolver, FileSubclusterResolver fileResolver)
+          throws IOException {
+    super(RouterRpcServer.class.getName());
+
+    this.rpcServer = null;
+  }
+
+  /**
+   * Allow access to the client RPC server for testing.
+   *
+   * @return The RPC server.
+   */
+  @VisibleForTesting
+  public Server getServer() {
+    return this.rpcServer;
+  }
+
+  @Override
+  protected void serviceInit(Configuration configuration) throws Exception {
+    super.serviceInit(configuration);
+  }
+
+  /**
+   * Start client and service RPC servers.
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    if (this.rpcServer != null) {
+      this.rpcServer.start();
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (this.rpcServer != null) {
+      this.rpcServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  /**
+   * Wait until the RPC servers have shutdown.
+   */
+  void join() throws InterruptedException {
+    if (this.rpcServer != null) {
+      this.rpcServer.join();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
new file mode 100644
index 0000000..327f39b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/package-info.java
@@ -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.
+ */
+
+/**
+ * The router package includes the core services for a HDFS federation router.
+ * The {@link Router} acts as a transparent proxy in front of a cluster of
+ * multiple NameNodes and nameservices. The {@link RouterRpcServer} exposes the
+ * NameNode clientProtocol and is the primary contact point for DFS clients in a
+ * federated cluster.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
new file mode 100644
index 0000000..866daa3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.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.hdfs.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.CompositeService;
+
+/**
+ * A service to initialize a
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} and maintain the connection to the data store. There
+ * are multiple state store driver connections supported:
+ * <ul>
+ * <li>File {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * StateStoreFileImpl StateStoreFileImpl}
+ * <li>ZooKeeper {@link org.apache.hadoop.hdfs.server.federation.store.driver.
+ * impl.StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
+ * </ul>
+ * <p>
+ * The service also supports the dynamic registration of data interfaces such as
+ * the following:
+ * <ul>
+ * <li>{@link MembershipStateStore}: state of the Namenodes in the
+ * federation.
+ * <li>{@link MountTableStore}: Mount table between to subclusters.
+ * See {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
+ * <li>{@link RouterStateStore}: State of the routers in the federation.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class StateStoreService extends CompositeService {
+
+  /** Identifier for the service. */
+  private String identifier;
+
+  // Stub class
+  public StateStoreService(String name) {
+    super(name);
+  }
+
+  /**
+   * Fetch a unique identifier for this state store instance. Typically it is
+   * the address of the router.
+   *
+   * @return Unique identifier for this store.
+   */
+  public String getIdentifier() {
+    return this.identifier;
+  }
+
+  /**
+   * Set a unique synchronization identifier for this store.
+   *
+   * @param id Unique identifier, typically the router's RPC address.
+   */
+  public void setIdentifier(String id) {
+    this.identifier = id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
new file mode 100644
index 0000000..949ec7c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/package-info.java
@@ -0,0 +1,62 @@
+/**
+ * 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.
+ */
+
+/**
+ * The federation state store tracks persistent values that are shared between
+ * multiple routers.
+ * <p>
+ * Data is stored in data records that inherit from a common class. Data records
+ * are serialized when written to the data store using a modular serialization
+ * implementation. The default is profobuf serialization. Data is stored as rows
+ * of records of the same type with each data member in a record representing a
+ * column.
+ * <p>
+ * The state store uses a modular data storage
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} to handle querying, updating and deleting data records. The
+ * data storage driver is initialized and maintained by the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.
+ * StateStoreService FederationStateStoreService}. The state store
+ * supports fetching all records of a type, filtering by column values or
+ * fetching a single record by its primary key.
+ * <p>
+ * The state store contains several API interfaces, one for each data records
+ * type.
+ * <p>
+ * <ul>
+ * <li>FederationMembershipStateStore: state of all Namenodes in the federation.
+ * Uses the MembershipState record.
+ * <li>FederationMountTableStore: Mount table mapping paths in the global
+ * namespace to individual subcluster paths. Uses the MountTable record.
+ * <li>RouterStateStore: State of all routers in the federation. Uses the
+ * RouterState record.
+ * </ul>
+ * <p>
+ * Each API is defined in a separate interface. The implementations of these
+ * interfaces are responsible for accessing the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} to query, update and delete data records.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 4caee9e..4a4dc11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4543,4 +4543,20 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.file.resolver.client.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <description>
+      Class to resolve files to subclusters.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.namenode.resolver.client.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <description>
+      Class to resolve the namenode for a subcluster.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
new file mode 100644
index 0000000..8674682
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
@@ -0,0 +1,233 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Date;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.security.AccessControlException;
+
+/**
+ * Helper utilities for testing HDFS Federation.
+ */
+public final class FederationTestUtils {
+
+  public final static String NAMESERVICE1 = "ns0";
+  public final static String NAMESERVICE2 = "ns1";
+  public final static String NAMENODE1 = "nn0";
+  public final static String NAMENODE2 = "nn1";
+  public final static String NAMENODE3 = "nn2";
+  public final static String NAMENODE4 = "nn3";
+  public final static String ROUTER1 = "router0";
+  public final static String ROUTER2 = "router1";
+  public final static String ROUTER3 = "router2";
+  public final static String ROUTER4 = "router3";
+  public final static long BLOCK_SIZE_BYTES = 134217728;
+
+  private FederationTestUtils() {
+    // Utility class
+  }
+
+  public static void verifyException(Object obj, String methodName,
+      Class<? extends Exception> exceptionClass, Class<?>[] parameterTypes,
+      Object[] arguments) {
+
+    Throwable triggeredException = null;
+    try {
+      Method m = obj.getClass().getMethod(methodName, parameterTypes);
+      m.invoke(obj, arguments);
+    } catch (InvocationTargetException ex) {
+      triggeredException = ex.getTargetException();
+    } catch (Exception e) {
+      triggeredException = e;
+    }
+    if (exceptionClass != null) {
+      assertNotNull("No exception was triggered, expected exception - "
+          + exceptionClass.getName(), triggeredException);
+      assertEquals(exceptionClass, triggeredException.getClass());
+    } else {
+      assertNull("Exception was triggered but no exception was expected",
+          triggeredException);
+    }
+  }
+
+  public static NamenodeStatusReport createNamenodeReport(String ns, String nn,
+      HAServiceState state) {
+    Random rand = new Random();
+    NamenodeStatusReport report = new NamenodeStatusReport(ns, nn,
+        "localhost:" + rand.nextInt(10000), "localhost:" + rand.nextInt(10000),
+        "localhost:" + rand.nextInt(10000), "testwebaddress-" + ns + nn);
+    if (state == null) {
+      // Unavailable, no additional info
+      return report;
+    }
+    report.setHAServiceState(state);
+    report.setNamespaceInfo(new NamespaceInfo(1, "tesclusterid", ns, 0,
+            "testbuildvesion", "testsoftwareversion"));
+    return report;
+  }
+
+  public static void waitNamenodeRegistered(ActiveNamenodeResolver resolver,
+      String nameserviceId, String namenodeId,
+      FederationNamenodeServiceState finalState)
+          throws InterruptedException, IllegalStateException, IOException {
+
+    for (int loopCount = 0; loopCount < 20; loopCount++) {
+
+      if (loopCount > 0) {
+        Thread.sleep(1000);
+      }
+
+      List<? extends FederationNamenodeContext> namenodes;
+      namenodes =
+          resolver.getNamenodesForNameserviceId(nameserviceId);
+      for (FederationNamenodeContext namenode : namenodes) {
+        if (namenodeId != null
+            && !namenode.getNamenodeId().equals(namenodeId)) {
+          // Keep looking
+          continue;
+        }
+        if (finalState != null && !namenode.getState().equals(finalState)) {
+          // Wrong state, wait a bit more
+          break;
+        }
+        // Found
+        return;
+      }
+    }
+    assertTrue("Failed to verify state store registration for state - "
+        + finalState + " - " + " - " + nameserviceId + " - ", false);
+  }
+
+  public static boolean verifyDate(Date d1, Date d2, long precision) {
+    if (Math.abs(d1.getTime() - d2.getTime()) < precision) {
+      return true;
+    }
+    return false;
+  }
+
+  public static boolean addDirectory(FileSystem context, String path)
+      throws IOException {
+    context.mkdirs(new Path(path), new FsPermission("777"));
+    return verifyFileExists(context, path);
+  }
+
+  public static FileStatus getFileStatus(FileSystem context, String path)
+      throws IOException {
+    return context.getFileStatus(new Path(path));
+  }
+
+  public static boolean verifyFileExists(FileSystem context, String path) {
+    try {
+      FileStatus status = getFileStatus(context, path);
+      if (status != null) {
+        return true;
+      }
+    } catch (Exception e) {
+      return false;
+    }
+
+    return false;
+  }
+
+  public static boolean checkForFileInDirectory(FileSystem context,
+      String testPath, String targetFile) throws AccessControlException,
+          FileNotFoundException,
+          UnsupportedFileSystemException, IllegalArgumentException,
+          IOException {
+    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    String file = null;
+    String verifyPath = testPath + "/" + targetFile;
+    if (testPath.equals("/")) {
+      verifyPath = testPath + targetFile;
+    }
+
+    Boolean found = false;
+    for (int i = 0; i < fileStatus.length; i++) {
+      FileStatus f = fileStatus[i];
+      file = Path.getPathWithoutSchemeAndAuthority(f.getPath()).toString();
+      if (file.equals(verifyPath)) {
+        found = true;
+      }
+    }
+    return found;
+  }
+
+  public static int countContents(FileSystem context, String testPath)
+      throws IOException {
+    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    return fileStatus.length;
+  }
+
+  public static void createFile(FileSystem fs, String path, long length)
+      throws IOException {
+    FsPermission permissions = new FsPermission("700");
+    FSDataOutputStream writeStream = fs.create(new Path(path), permissions,
+        true, 1000, (short) 1, BLOCK_SIZE_BYTES, null);
+    for (int i = 0; i < length; i++) {
+      writeStream.write(i);
+    }
+    writeStream.close();
+  }
+
+  public static String readFile(FileSystem fs, String path) throws IOException {
+    // Read the file from the filesystem via the active namenode
+    Path fileName = new Path(path);
+    InputStreamReader reader = new InputStreamReader(fs.open(fileName));
+    BufferedReader bufferedReader = new BufferedReader(reader);
+    StringBuilder data = new StringBuilder();
+    String line;
+
+    while ((line = bufferedReader.readLine()) != null) {
+      data.append(line);
+    }
+
+    bufferedReader.close();
+    reader.close();
+    return data.toString();
+  }
+
+  public static boolean deleteFile(FileSystem fs, String path)
+      throws IOException {
+    return fs.delete(new Path(path), true);
+  }
+}


---------------------------------------------------------------------
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: HDFS-2319. Add test cases for FSshell -stat. Contributed by XieXianshan and Bharat Viswanadham.

Posted by in...@apache.org.
HDFS-2319. Add test cases for FSshell -stat. Contributed by XieXianshan and Bharat Viswanadham.


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

Branch: refs/heads/HDFS-10467
Commit: e3c73002250a21a771689081b51764eca1d862a7
Parents: 5f4808c
Author: Jitendra Pandey <ji...@apache.org>
Authored: Thu Jul 27 13:23:15 2017 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Thu Jul 27 13:23:15 2017 -0700

----------------------------------------------------------------------
 .../src/test/resources/testHDFSConf.xml         | 125 ++++++++++++++++++-
 1 file changed, 124 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3c73002/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 9302507..ba90efa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -7203,7 +7203,130 @@
         </comparator>
       </comparators>
     </test>
-  
+
+    <test> <!-- TESTED -->
+      <description>stat: Test for hdfs:// path - user/group name for directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir hdfs:///dirtest</command>
+        <command>-fs NAMENODE -chown hadoop:hadoopgrp hdfs:///dirtest</command>
+        <command>-fs NAMENODE -stat "%u-%g" hdfs:///dirtest</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r hdfs:///dirtest</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hadoop-hadoopgrp</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>stat: Test for hdfs:// path - user/group name for file</description>
+      <test-commands>
+        <command>-fs NAMENODE -put CLITEST_DATA/data60bytes hdfs:///data60bytes</command>
+        <command>-fs NAMENODE -chown hadoop:hadoopgrp hdfs:////data60bytes</command>
+        <command>-fs NAMENODE -stat "%u-%g" hdfs:////data60bytes</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r hdfs:///data60bytes</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hadoop-hadoopgrp</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>stat: Test for hdfs:// path - user/group name for multiple files</description>
+      <test-commands>
+        <command>-fs NAMENODE -put CLITEST_DATA/data60bytes hdfs:///data60bytes</command>
+        <command>-fs NAMENODE -put CLITEST_DATA/data30bytes hdfs:///data30bytes</command>
+        <command>-fs NAMENODE -chown hadoop:hadoopgrp hdfs:///data60bytes</command>
+        <command>-fs NAMENODE -chown hdfs:hdfs hdfs:///data30bytes</command>
+        <command>-fs NAMENODE -stat "%u-%g" hdfs:///data*</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r hdfs:///data60bytes</command>
+        <command>-fs NAMENODE -rm -r hdfs:////data30bytes</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hadoop-hadoopgrp</expected-output>
+        </comparator>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hdfs-hdfs</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>stat: Test for Namenode's path - user/group name for directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir0</command>
+        <command>-fs NAMENODE -chown hadoop:hadoopgrp NAMENODE/dir0/</command>
+        <command>-fs NAMENODE -stat "%u-%g" NAMENODE/dir0/</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r NAMENODE/dir0</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hadoop-hadoopgrp</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+    <description>stat: Test for Namenode's path - user/group name for file </description>
+    <test-commands>
+    <command>-fs NAMENODE -mkdir /dir0</command>
+    <command>-fs NAMENODE -put CLITEST_DATA/data15bytes NAMENODE/dir0/data15bytes</command>
+    <command>-fs NAMENODE -chown hadoop:hadoopgrp NAMENODE/dir0/data15bytes</command>
+    <command>-fs NAMENODE -stat "%u-%g" NAMENODE/dir0/data15bytes</command>
+    </test-commands>
+    <cleanup-commands>
+      <command>-fs NAMENODE -rm -r NAMENODE/dir0</command>
+    </cleanup-commands>
+    <comparators>
+      <comparator>
+        <type>TokenComparator</type>
+        <expected-output>hadoop-hadoopgrp</expected-output>
+      </comparator>
+    </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>stat: Test for Namenode's path - user/group name for multiple files </description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /dir0</command>
+        <command>-fs NAMENODE -put CLITEST_DATA/data15bytes NAMENODE/dir0/data15bytes</command>
+        <command>-fs NAMENODE -put CLITEST_DATA/data30bytes NAMENODE/dir0/data30bytes</command>
+        <command>-fs NAMENODE -chown hadoop:hadoopgrp NAMENODE/dir0/data15bytes</command>
+        <command>-fs NAMENODE -chown hdfs:hdfs NAMENODE/dir0/data30bytes</command>
+        <command>-fs NAMENODE -stat "%u-%g" NAMENODE/dir0/data*</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r NAMENODE/dir0</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hadoop-hadoopgrp</expected-output>
+        </comparator>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>hdfs-hdfs</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
     <!-- Tests for tail -->
     <test> <!-- TESTED -->
       <description>tail: contents of file(absolute path)</description>


---------------------------------------------------------------------
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-6804. Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi

Posted by in...@apache.org.
YARN-6804. Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi


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

Branch: refs/heads/HDFS-10467
Commit: ac9489f7fc2dd351fbe5be4b7a3add4782da81c3
Parents: a68b5b3
Author: Jian He <ji...@apache.org>
Authored: Mon Jul 24 21:08:10 2017 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Jul 25 09:57:14 2017 -0700

----------------------------------------------------------------------
 .../hadoop-client-minicluster/pom.xml           |  4 ++
 .../client/binding/RegistryPathUtils.java       |  2 +-
 .../hadoop/registry/client/types/Endpoint.java  |  4 +-
 .../registry/client/types/ServiceRecord.java    |  4 +-
 .../hadoop-yarn-server-nodemanager/pom.xml      |  4 ++
 .../runtime/DockerLinuxContainerRuntime.java    | 67 +++++++++++++++-----
 .../linux/runtime/docker/DockerRunCommand.java  |  6 ++
 .../impl/container-executor.c                   |  4 ++
 .../test/test-container-executor.c              | 16 ++---
 .../runtime/TestDockerContainerRuntime.java     | 58 +++++++++++++----
 10 files changed, 125 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index 4512906..93811ad 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -395,6 +395,10 @@
         </exclusion>
         <exclusion>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-registry</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-yarn-server-common</artifactId>
         </exclusion>
         <exclusion>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
index 5d8ea3f..5fa45f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
@@ -213,6 +213,6 @@ public class RegistryPathUtils {
    * @return a string suitable for use in registry paths.
    */
   public static String encodeYarnID(String yarnId) {
-    return yarnId.replace("_", "-");
+    return yarnId.replace("container", "ctr").replace("_", "-");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
index 395f836..392884f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -46,7 +46,7 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public final class Endpoint implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
index 674d6d3..d40866a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonAnyGetter;
 import com.fasterxml.jackson.annotation.JsonAnySetter;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -37,7 +37,7 @@ import java.util.Map;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public class ServiceRecord implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index a0f4ef7..094519a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -52,6 +52,10 @@
       <artifactId>hadoop-yarn-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+    </dependency>
+    <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.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/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 8db03bc..e058d6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
@@ -101,6 +102,11 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  *     property.
  *   </li>
  *   <li>
+ *     {@code YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME} sets the
+ *     hostname to be used by the Docker container. If not specified, a
+ *     hostname will be derived from the container ID.
+ *   </li>
+ *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER}
  *     controls whether the Docker container is a privileged container. In order
  *     to use privileged containers, the
@@ -134,6 +140,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       "^(([a-zA-Z0-9.-]+)(:\\d+)?/)?([a-z0-9_./-]+)(:[\\w.-]+)?$";
   private static final Pattern dockerImagePattern =
       Pattern.compile(DOCKER_IMAGE_PATTERN);
+  public static final String HOSTNAME_PATTERN =
+      "^[a-zA-Z0-9][a-zA-Z0-9_.-]+$";
+  private static final Pattern hostnamePattern = Pattern.compile(
+      HOSTNAME_PATTERN);
 
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_IMAGE =
@@ -147,6 +157,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_NETWORK =
       "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
+  @InterfaceAudience.Private
+  public static final String ENV_DOCKER_CONTAINER_HOSTNAME =
+      "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME";
+  @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER =
       "YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER";
   @InterfaceAudience.Private
@@ -211,9 +225,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     this.privilegedOperationExecutor = privilegedOperationExecutor;
 
     if (cGroupsHandler == null) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("cGroupsHandler is null - cgroups not in use.");
-      }
+      LOG.info("cGroupsHandler is null - cgroups not in use.");
     } else {
       this.cGroupsHandler = cGroupsHandler;
     }
@@ -267,6 +279,29 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     throw new ContainerExecutionException(msg);
   }
 
+  public static void validateHostname(String hostname) throws
+      ContainerExecutionException {
+    if (hostname != null && !hostname.isEmpty()) {
+      if (!hostnamePattern.matcher(hostname).matches()) {
+        throw new ContainerExecutionException("Hostname '" + hostname
+            + "' doesn't match docker hostname pattern");
+      }
+    }
+  }
+
+  /** Set a DNS friendly hostname. */
+  private void setHostname(DockerRunCommand runCommand, String
+      containerIdStr, String name)
+      throws ContainerExecutionException {
+    if (name == null || name.isEmpty()) {
+      name = RegistryPathUtils.encodeYarnID(containerIdStr);
+      validateHostname(name);
+    }
+
+    LOG.info("setting hostname in container to: " + name);
+    runCommand.setHostname(name);
+  }
+
   /**
    * If CGROUPS in enabled and not set to none, then set the CGROUP parent for
    * the command instance.
@@ -343,10 +378,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       return false;
     }
 
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Privileged container requested for : " + container
-          .getContainerId().toString());
-    }
+    LOG.info("Privileged container requested for : " + container
+        .getContainerId().toString());
 
     //Ok, so we have been asked to run a privileged container. Security
     // checks need to be run. Each violation is an error.
@@ -375,10 +408,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throw new ContainerExecutionException(message);
     }
 
-    if (LOG.isInfoEnabled()) {
-      LOG.info("All checks pass. Launching privileged container for : "
-          + container.getContainerId().toString());
-    }
+    LOG.info("All checks pass. Launching privileged container for : "
+        + container.getContainerId().toString());
 
     return true;
   }
@@ -413,6 +444,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         .getEnvironment();
     String imageName = environment.get(ENV_DOCKER_CONTAINER_IMAGE);
     String network = environment.get(ENV_DOCKER_CONTAINER_NETWORK);
+    String hostname = environment.get(ENV_DOCKER_CONTAINER_HOSTNAME);
 
     if(network == null || network.isEmpty()) {
       network = defaultNetwork;
@@ -420,6 +452,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
 
     validateContainerNetworkType(network);
 
+    validateHostname(hostname);
+
     validateImageName(imageName);
 
     String containerIdStr = container.getContainerId().toString();
@@ -450,12 +484,13 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         runAsUser, imageName)
         .detachOnRun()
         .setContainerWorkDir(containerWorkDir.toString())
-        .setNetworkType(network)
-        .setCapabilities(capabilities)
+        .setNetworkType(network);
+    setHostname(runCommand, containerIdStr, hostname);
+    runCommand.setCapabilities(capabilities)
         .addMountLocation(CGROUPS_ROOT_DIRECTORY,
             CGROUPS_ROOT_DIRECTORY + ":ro", false);
-    List<String> allDirs = new ArrayList<>(containerLocalDirs);
 
+    List<String> allDirs = new ArrayList<>(containerLocalDirs);
     allDirs.addAll(filecacheDirs);
     allDirs.add(containerWorkDir.toString());
     allDirs.addAll(containerLogDirs);
@@ -493,9 +528,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         ENV_DOCKER_CONTAINER_RUN_OVERRIDE_DISABLE);
 
     if (disableOverride != null && disableOverride.equals("true")) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("command override disabled");
-      }
+      LOG.info("command override disabled");
     } else {
       List<String> overrideCommands = new ArrayList<>();
       Path launchDst =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.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/linux/runtime/docker/DockerRunCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
index f79f4ed..b645754 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
@@ -91,6 +91,12 @@ public class DockerRunCommand extends DockerCommand {
 
     return this;
   }
+
+  public DockerRunCommand setHostname(String hostname) {
+    super.addCommandArguments("--hostname=" + hostname);
+    return this;
+  }
+
   public DockerRunCommand addDevice(String sourceDevice, String
       destinationDevice) {
     super.addCommandArguments("--device=" + sourceDevice + ":" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 5d138f3..5070d62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1215,6 +1215,7 @@ char* sanitize_docker_command(const char *line) {
     {"rm", no_argument, 0, 'r' },
     {"workdir", required_argument, 0, 'w' },
     {"net", required_argument, 0, 'e' },
+    {"hostname", required_argument, 0, 'h' },
     {"cgroup-parent", required_argument, 0, 'g' },
     {"privileged", no_argument, 0, 'p' },
     {"cap-add", required_argument, 0, 'a' },
@@ -1256,6 +1257,9 @@ char* sanitize_docker_command(const char *line) {
       case 'e':
         quote_and_append_arg(&output, &output_size, "--net=", optarg);
         break;
+      case 'h':
+        quote_and_append_arg(&output, &output_size, "--hostname=", optarg);
+        break;
       case 'v':
         quote_and_append_arg(&output, &output_size, "-v ", optarg);
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index 83d11ec..b7d0e44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -1088,17 +1088,17 @@ void test_trim_function() {
 void test_sanitize_docker_command() {
 
   char *input[] = {
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
     "run ''''''''"
   };
   char *expected_output[] = {
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
       "run ''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"'' ",
   };
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac9489f7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.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/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index f611843..9894dcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -69,6 +70,7 @@ public class TestDockerContainerRuntime {
   private PrivilegedOperationExecutor mockExecutor;
   private CGroupsHandler mockCGroupsHandler;
   private String containerId;
+  private String defaultHostname;
   private Container container;
   private ContainerId cId;
   private ContainerLaunchContext context;
@@ -108,6 +110,7 @@ public class TestDockerContainerRuntime {
         .mock(PrivilegedOperationExecutor.class);
     mockCGroupsHandler = Mockito.mock(CGroupsHandler.class);
     containerId = "container_id";
+    defaultHostname = RegistryPathUtils.encodeYarnID(containerId);
     container = mock(Container.class);
     cId = mock(ContainerId.class);
     context = mock(ContainerLaunchContext.class);
@@ -287,6 +290,7 @@ public class TestDockerContainerRuntime {
         .append("--user=%2$s -d ")
         .append("--workdir=%3$s ")
         .append("--net=host ")
+        .append("--hostname=" + defaultHostname + " ")
         .append(getExpectedTestCapabilitiesArgumentString())
         .append(getExpectedCGroupsMountString())
         .append("-v %4$s:%4$s ")
@@ -365,7 +369,7 @@ public class TestDockerContainerRuntime {
     String disallowedNetwork = "sdn" + Integer.toString(randEngine.nextInt());
 
     try {
-      env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+      env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
           disallowedNetwork);
       runtime.launchContainer(builder.build());
       Assert.fail("Network was expected to be disallowed: " +
@@ -378,8 +382,11 @@ public class TestDockerContainerRuntime {
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS.length;
     String allowedNetwork = YarnConfiguration
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS[randEngine.nextInt(size)];
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         allowedNetwork);
+    String expectedHostname = "test.hostname";
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_HOSTNAME,
+        expectedHostname);
 
     //this should cause no failures.
 
@@ -393,6 +400,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + allowedNetwork + " ")
+            .append("--hostname=" + expectedHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -448,6 +456,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork1 + " ")
+            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -471,7 +480,7 @@ public class TestDockerContainerRuntime {
     //now set an explicit (non-default) allowedNetwork and ensure that it is
     // used.
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         customNetwork2);
     runtime.launchContainer(builder.build());
 
@@ -485,6 +494,7 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork2 + " ")
+            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -505,7 +515,7 @@ public class TestDockerContainerRuntime {
 
     //disallowed network should trigger a launch failure
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
+    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
         customNetwork3);
     try {
       runtime.launchContainer(builder.build());
@@ -524,8 +534,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "invalid-value");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
     runtime.launchContainer(builder.build());
 
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -552,8 +562,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -575,8 +585,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
     //By default
     // yarn.nodemanager.runtime.linux.docker.privileged-containers.acl
     // is empty. So we expect this launch to fail.
@@ -605,8 +615,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -632,8 +642,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
-        "true");
+    env.put(DockerLinuxContainerRuntime
+            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
 
     runtime.launchContainer(builder.build());
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -927,4 +937,24 @@ public class TestDockerContainerRuntime {
       }
     }
   }
+
+  @Test
+  public void testDockerHostnamePattern() throws Exception {
+    String[] validNames = {"ab", "a.b.c.d", "a1-b.cd.ef", "0AB.", "C_D-"};
+
+    String[] invalidNames = {"a", "a#.b.c", "-a.b.c", "a@b.c", "a/b/c"};
+
+    for (String name : validNames) {
+      DockerLinuxContainerRuntime.validateHostname(name);
+    }
+
+    for (String name : invalidNames) {
+      try {
+        DockerLinuxContainerRuntime.validateHostname(name);
+        Assert.fail(name + " is an invalid hostname and should fail the regex");
+      } catch (ContainerExecutionException ce) {
+        continue;
+      }
+    }
+  }
 }


---------------------------------------------------------------------
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-12094. Log torrent when none isa-l EC is used. Contributed by LiXin Ge.

Posted by in...@apache.org.
HDFS-12094. Log torrent when none isa-l EC is used. 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/465c2134
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/465c2134
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/465c2134

Branch: refs/heads/HDFS-10467
Commit: 465c213439d4955df0b23d38423073890b121ccc
Parents: 4a771d9
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Jul 21 12:24:09 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Jul 21 12:25:03 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/io/erasurecode/CodecUtil.java   | 14 +++++++++-----
 .../hadoop/io/erasurecode/ErasureCodeNative.java      |  5 +++++
 2 files changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/465c2134/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
index 69df56a..8ec0e72 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
@@ -178,8 +178,10 @@ public final class CodecUtil {
         }
       } catch (LinkageError | Exception e) {
         // Fallback to next coder if possible
-        LOG.warn("Failed to create raw erasure encoder " + rawCoderName +
-            ", fallback to next codec if possible", e);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Failed to create raw erasure encoder " + rawCoderName +
+              ", fallback to next codec if possible", e);
+        }
       }
     }
     throw new IllegalArgumentException("Fail to create raw erasure " +
@@ -198,12 +200,14 @@ public final class CodecUtil {
         }
       } catch (LinkageError | Exception e) {
         // Fallback to next coder if possible
-        LOG.warn("Failed to create raw erasure decoder " + rawCoderName +
-            ", fallback to next codec if possible", e);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Failed to create raw erasure decoder " + rawCoderName +
+                  ", fallback to next codec if possible", e);
+        }
       }
     }
     throw new IllegalArgumentException("Fail to create raw erasure " +
-        "encoder with given codec: " + codecName);
+        "decoder with given codec: " + codecName);
   }
 
   private static ErasureCodec createCodec(Configuration conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/465c2134/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java
index cb462b8..3d6867a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java
@@ -50,6 +50,11 @@ public final class ErasureCodeNative {
       }
       LOADING_FAILURE_REASON = problem;
     }
+
+    if (LOADING_FAILURE_REASON != null) {
+      LOG.warn("ISA-L support is not available in your platform... " +
+              "using builtin-java codec where applicable");
+    }
   }
 
   private ErasureCodeNative() {}


---------------------------------------------------------------------
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: HADOOP-14681. Remove MockitoMaker class. Contributed by Andras Bokor.

Posted by in...@apache.org.
HADOOP-14681. Remove MockitoMaker class. 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/cca51e91
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cca51e91
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cca51e91

Branch: refs/heads/HDFS-10467
Commit: cca51e916b7387ea358688e8f8188ead948fbdcc
Parents: 218b1b3
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jul 25 15:24:56 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jul 25 15:24:56 2017 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/test/MockitoMaker.java    | 132 -------------------
 .../org/apache/hadoop/util/TestDiskChecker.java |  12 +-
 .../v2/app/metrics/TestMRAppMetrics.java        |   9 +-
 .../lib/input/TestMRCJCFileInputFormat.java     |  11 +-
 .../hadoop/mapred/TestShuffleHandler.java       |  32 ++---
 .../scheduler/TestQueueMetrics.java             |  14 +-
 6 files changed, 38 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoMaker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoMaker.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoMaker.java
deleted file mode 100644
index 28c2011..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MockitoMaker.java
+++ /dev/null
@@ -1,132 +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.test;
-
-import static org.mockito.Mockito.*;
-
-/**
- * Helper class to create one-liner stubs, so that instead of: <pre>
- * SomeType someDescriptiveMock = mock(SomeType.class);
- * when(someDescriptiveMock.someMethod()).thenReturn(someValue);</pre>
- * <p>You can now do: <pre>
- * SomeType someDescriptiveMock = make(stub(SomeType.class)
- *     .returning(someValue).from.someMethod());</pre>
- */
-public class MockitoMaker {
-
-  /**
-   * Create a mock object from a mocked method call.
-   *
-   * @param <T> type of mocked object
-   * @param methodCall  for mocked object
-   * @return mocked object
-   */
-  @SuppressWarnings("unchecked")
-  public static <T> T make(Object methodCall) {
-    StubBuilder<T> sb = StubBuilder.current();
-    when(methodCall).thenReturn(sb.firstReturn, sb.laterReturns);
-    return (T) StubBuilder.current().from;
-  }
-
-  /**
-   * Create a stub builder of a mocked object.
-   *
-   * @param <T>     type of the target object to be mocked
-   * @param target  class of the target object to be mocked
-   * @return the stub builder of the mocked object
-   */
-  public static <T> StubBuilder<T> stub(Class<T> target) {
-    return new StubBuilder<T>(mock(target));
-  }
-
-  /**
-   * Builder class for stubs
-   * @param <T> type of the object to be mocked
-   */
-  public static class StubBuilder<T> {
-
-    /**
-     * The target mock object
-     */
-    public final T from;
-
-    // We want to be able to use this even when the tests are run in parallel.
-    @SuppressWarnings("rawtypes")
-    private static final ThreadLocal<StubBuilder> tls =
-        new ThreadLocal<StubBuilder>() {
-          @Override protected StubBuilder initialValue() {
-            return new StubBuilder();
-          }
-        };
-
-    private Object firstReturn = null;
-    private Object[] laterReturns = {};
-
-    /**
-     * Default constructor for the initial stub builder
-     */
-    public StubBuilder() {
-      this.from = null;
-    }
-
-    /**
-     * Construct a stub builder with a mock instance
-     *
-     * @param mockInstance  the mock object
-     */
-    public StubBuilder(T mockInstance) {
-      tls.set(this);
-      this.from = mockInstance;
-    }
-
-    /**
-     * Get the current stub builder from thread local
-     *
-     * @param <T>
-     * @return the stub builder of the mocked object
-     */
-    @SuppressWarnings("unchecked")
-    public static <T> StubBuilder<T> current() {
-      return tls.get();
-    }
-
-    /**
-     * Set the return value for the current stub builder
-     *
-     * @param value the return value
-     * @return the stub builder
-     */
-    public StubBuilder<T> returning(Object value) {
-      this.firstReturn = value;
-      return this;
-    }
-
-    /**
-     * Set the return values for the current stub builder
-     *
-     * @param value   the first return value
-     * @param values  the return values for later invocations
-     * @return the stub builder
-     */
-    public StubBuilder<T> returning(Object value, Object... values) {
-      this.firstReturn = value;
-      this.laterReturns = values;
-      return this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
index 40b8d0d..bd8e1dd 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskChecker.java
@@ -29,7 +29,6 @@ import static org.junit.Assert.*;
 
 import static org.mockito.Mockito.*;
 
-import static org.apache.hadoop.test.MockitoMaker.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -83,13 +82,14 @@ public class TestDiskChecker {
 
   private void _mkdirs(boolean exists, FsPermission before, FsPermission after)
       throws Throwable {
-    File localDir = make(stub(File.class).returning(exists).from.exists());
+    File localDir = mock(File.class);
+    when(localDir.exists()).thenReturn(exists);
     when(localDir.mkdir()).thenReturn(true);
     Path dir = mock(Path.class); // use default stubs
-    LocalFileSystem fs = make(stub(LocalFileSystem.class)
-        .returning(localDir).from.pathToFile(dir));
-    FileStatus stat = make(stub(FileStatus.class)
-        .returning(after).from.getPermission());
+    LocalFileSystem fs = mock(LocalFileSystem.class);
+    when(fs.pathToFile(dir)).thenReturn(localDir);
+    FileStatus stat = mock(FileStatus.class);
+    when(stat.getPermission()).thenReturn(after);
     when(fs.getFileStatus(dir)).thenReturn(stat);
 
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
index 8bfc2a8..02552bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 
 import static org.apache.hadoop.test.MetricsAsserts.*;
-import static org.apache.hadoop.test.MockitoMaker.*;
 
 import org.junit.Test;
 
@@ -33,10 +32,10 @@ public class TestMRAppMetrics {
 
   @Test public void testNames() {
     Job job = mock(Job.class);
-    Task mapTask = make(stub(Task.class).returning(TaskType.MAP).
-                        from.getType());
-    Task reduceTask = make(stub(Task.class).returning(TaskType.REDUCE).
-                           from.getType());
+    Task mapTask = mock(Task.class);
+    when(mapTask.getType()).thenReturn(TaskType.MAP);
+    Task reduceTask = mock(Task.class);
+    when(reduceTask.getType()).thenReturn(TaskType.REDUCE);
     MRAppMetrics metrics = MRAppMetrics.create();
 
     metrics.submittedJob(job);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java
index b806630..ef25876 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMRCJCFileInputFormat.java
@@ -27,7 +27,6 @@ import org.junit.Test;
 import static org.junit.Assert.*;
 
 import static org.mockito.Mockito.*;
-import static org.apache.hadoop.test.MockitoMaker.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -80,12 +79,14 @@ public class TestMRCJCFileInputFormat {
   @Test
   public void testNumInputFiles() throws Exception {
     Configuration conf = spy(new Configuration());
-    Job job = make(stub(Job.class).returning(conf).from.getConfiguration());
-    FileStatus stat = make(stub(FileStatus.class).returning(0L).from.getLen());
+    Job mockedJob = mock(Job.class);
+    when(mockedJob.getConfiguration()).thenReturn(conf);
+    FileStatus stat = mock(FileStatus.class);
+    when(stat.getLen()).thenReturn(0L);
     TextInputFormat ispy = spy(new TextInputFormat());
-    doReturn(Arrays.asList(stat)).when(ispy).listStatus(job);
+    doReturn(Arrays.asList(stat)).when(ispy).listStatus(mockedJob);
 
-    ispy.getSplits(job);
+    ispy.getSplits(mockedJob);
     verify(conf).setLong(FileInputFormat.NUM_INPUT_FILES, 1);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
index 7fb2051..849ce1a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
@@ -20,14 +20,14 @@ package org.apache.hadoop.mapred;
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.apache.hadoop.test.MockitoMaker.make;
-import static org.apache.hadoop.test.MockitoMaker.stub;
 import static org.junit.Assert.assertTrue;
 import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
 import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
 import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.DataInputStream;
 import java.io.EOFException;
@@ -188,8 +188,8 @@ public class TestShuffleHandler {
   public void testShuffleMetrics() throws Exception {
     MetricsSystem ms = new MetricsSystemImpl();
     ShuffleHandler sh = new ShuffleHandler(ms);
-    ChannelFuture cf = make(stub(ChannelFuture.class).
-        returning(true, false).from.isSuccess());
+    ChannelFuture cf = mock(ChannelFuture.class);
+    when(cf.isSuccess()).thenReturn(true).thenReturn(false);
 
     sh.metrics.shuffleConnections.incr();
     sh.metrics.shuffleOutputBytes.incr(1*MiB);
@@ -1080,10 +1080,10 @@ public class TestShuffleHandler {
         new ArrayList<ShuffleHandler.ReduceMapFileCount>();
 
     final ChannelHandlerContext mockCtx =
-        Mockito.mock(ChannelHandlerContext.class);
-    final MessageEvent mockEvt = Mockito.mock(MessageEvent.class);
-    final Channel mockCh = Mockito.mock(AbstractChannel.class);
-    final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class);
+        mock(ChannelHandlerContext.class);
+    final MessageEvent mockEvt = mock(MessageEvent.class);
+    final Channel mockCh = mock(AbstractChannel.class);
+    final ChannelPipeline mockPipeline = mock(ChannelPipeline.class);
 
     // Mock HttpRequest and ChannelFuture
     final HttpRequest mockHttpRequest = createMockHttpRequest();
@@ -1094,16 +1094,16 @@ public class TestShuffleHandler {
 
     // Mock Netty Channel Context and Channel behavior
     Mockito.doReturn(mockCh).when(mockCtx).getChannel();
-    Mockito.when(mockCh.getPipeline()).thenReturn(mockPipeline);
-    Mockito.when(mockPipeline.get(
+    when(mockCh.getPipeline()).thenReturn(mockPipeline);
+    when(mockPipeline.get(
         Mockito.any(String.class))).thenReturn(timerHandler);
-    Mockito.when(mockCtx.getChannel()).thenReturn(mockCh);
+    when(mockCtx.getChannel()).thenReturn(mockCh);
     Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class));
-    Mockito.when(mockCh.write(Object.class)).thenReturn(mockFuture);
+    when(mockCh.write(Object.class)).thenReturn(mockFuture);
 
     //Mock MessageEvent behavior
     Mockito.doReturn(mockCh).when(mockEvt).getChannel();
-    Mockito.when(mockEvt.getChannel()).thenReturn(mockCh);
+    when(mockEvt.getChannel()).thenReturn(mockCh);
     Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage();
 
     final ShuffleHandler sh = new MockShuffleHandler();
@@ -1127,8 +1127,8 @@ public class TestShuffleHandler {
 
   public ChannelFuture createMockChannelFuture(Channel mockCh,
       final List<ShuffleHandler.ReduceMapFileCount> listenerList) {
-    final ChannelFuture mockFuture = Mockito.mock(ChannelFuture.class);
-    Mockito.when(mockFuture.getChannel()).thenReturn(mockCh);
+    final ChannelFuture mockFuture = mock(ChannelFuture.class);
+    when(mockFuture.getChannel()).thenReturn(mockCh);
     Mockito.doReturn(true).when(mockFuture).isSuccess();
     Mockito.doAnswer(new Answer() {
       @Override
@@ -1146,7 +1146,7 @@ public class TestShuffleHandler {
   }
 
   public HttpRequest createMockHttpRequest() {
-    HttpRequest mockHttpRequest = Mockito.mock(HttpRequest.class);
+    HttpRequest mockHttpRequest = mock(HttpRequest.class);
     Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod();
     Mockito.doAnswer(new Answer() {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cca51e91/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.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/TestQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
index 13144e9..196d4c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.apache.hadoop.test.MockitoMaker.make;
-import static org.apache.hadoop.test.MockitoMaker.stub;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -226,8 +224,8 @@ public class TestQueueMetrics {
 
     QueueMetrics parentMetrics =
       QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
-    Queue parentQueue = make(stub(Queue.class).returning(parentMetrics).
-        from.getMetrics());
+    Queue parentQueue = mock(Queue.class);
+    when(parentQueue.getMetrics()).thenReturn(parentMetrics);
     QueueMetrics metrics =
       QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
     MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
@@ -272,8 +270,8 @@ public class TestQueueMetrics {
 
     QueueMetrics parentMetrics =
       QueueMetrics.forQueue(ms, parentQueueName, null, true, conf);
-    Queue parentQueue = make(stub(Queue.class).returning(parentMetrics).
-        from.getMetrics());
+    Queue parentQueue = mock(Queue.class);
+    when(parentQueue.getMetrics()).thenReturn(parentMetrics);
     QueueMetrics metrics =
       QueueMetrics.forQueue(ms, leafQueueName, parentQueue, true, conf);
     MetricsSource parentQueueSource = queueSource(ms, parentQueueName);
@@ -359,8 +357,8 @@ public class TestQueueMetrics {
 
       QueueMetrics p1Metrics =
           QueueMetrics.forQueue(ms, p1, null, true, conf);
-      Queue parentQueue1 = make(stub(Queue.class).returning(p1Metrics).
-          from.getMetrics());
+      Queue parentQueue1 = mock(Queue.class);
+      when(parentQueue1.getMetrics()).thenReturn(p1Metrics);
       QueueMetrics metrics =
           QueueMetrics.forQueue(ms, leafQueueName, parentQueue1, true, conf);
 


---------------------------------------------------------------------
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: HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
new file mode 100644
index 0000000..7f0b36a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java
@@ -0,0 +1,483 @@
+/**
+ * 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.store.driver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.junit.AfterClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base tests for the driver. The particular implementations will use this to
+ * test their functionality.
+ */
+public class TestStateStoreDriverBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestStateStoreDriverBase.class);
+
+  private static StateStoreService stateStore;
+  private static Configuration conf;
+
+
+  /**
+   * Get the State Store driver.
+   * @return State Store driver.
+   */
+  protected StateStoreDriver getStateStoreDriver() {
+    return stateStore.getDriver();
+  }
+
+  @AfterClass
+  public static void tearDownCluster() {
+    if (stateStore != null) {
+      stateStore.stop();
+    }
+  }
+
+  /**
+   * Get a new State Store using this configuration.
+   *
+   * @param config Configuration for the State Store.
+   * @throws Exception If we cannot get the State Store.
+   */
+  public static void getStateStore(Configuration config) throws Exception {
+    conf = config;
+    stateStore = FederationStateStoreTestUtils.getStateStore(conf);
+  }
+
+  private <T extends BaseRecord> T generateFakeRecord(Class<T> recordClass)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    // TODO add record
+    return null;
+  }
+
+  /**
+   * Validate if a record is the same.
+   *
+   * @param original
+   * @param committed
+   * @param assertEquals Assert if the records are equal or just return.
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   */
+  private boolean validateRecord(
+      BaseRecord original, BaseRecord committed, boolean assertEquals)
+          throws IllegalArgumentException, IllegalAccessException {
+
+    boolean ret = true;
+
+    Map<String, Class<?>> fields = getFields(original);
+    for (String key : fields.keySet()) {
+      if (key.equals("dateModified") ||
+          key.equals("dateCreated") ||
+          key.equals("proto")) {
+        // Fields are updated/set on commit and fetch and may not match
+        // the fields that are initialized in a non-committed object.
+        continue;
+      }
+      Object data1 = getField(original, key);
+      Object data2 = getField(committed, key);
+      if (assertEquals) {
+        assertEquals("Field " + key + " does not match", data1, data2);
+      } else if (!data1.equals(data2)) {
+        ret = false;
+      }
+    }
+
+    long now = stateStore.getDriver().getTime();
+    assertTrue(
+        committed.getDateCreated() <= now && committed.getDateCreated() > 0);
+    assertTrue(committed.getDateModified() >= committed.getDateCreated());
+
+    return ret;
+  }
+
+  public static void removeAll(StateStoreDriver driver) throws IOException {
+    // TODO add records to remove
+  }
+
+  public <T extends BaseRecord> void testInsert(
+      StateStoreDriver driver, Class<T> recordClass)
+          throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    assertTrue(driver.removeAll(recordClass));
+    QueryResult<T> records = driver.get(recordClass);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert single
+    BaseRecord record = generateFakeRecord(recordClass);
+    driver.put(record, true, false);
+
+    // Verify
+    records = driver.get(recordClass);
+    assertEquals(1, records.getRecords().size());
+    validateRecord(record, records.getRecords().get(0), true);
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(recordClass);
+      insertList.add(newRecord);
+    }
+    driver.putAll(insertList, true, false);
+
+    // Verify
+    records = driver.get(recordClass);
+    assertEquals(11, records.getRecords().size());
+  }
+
+  public <T extends BaseRecord> void testFetchErrors(StateStoreDriver driver,
+      Class<T> clazz) throws IllegalAccessException, IOException {
+
+    // Fetch empty list
+    driver.removeAll(clazz);
+    QueryResult<T> result0 = driver.get(clazz);
+    assertNotNull(result0);
+    List<T> records0 = result0.getRecords();
+    assertEquals(records0.size(), 0);
+
+    // Insert single
+    BaseRecord record = generateFakeRecord(clazz);
+    assertTrue(driver.put(record, true, false));
+
+    // Verify
+    QueryResult<T> result1 = driver.get(clazz);
+    List<T> records1 = result1.getRecords();
+    assertEquals(1, records1.size());
+    validateRecord(record, records1.get(0), true);
+
+    // Test fetch single object with a bad query
+    final T fakeRecord = generateFakeRecord(clazz);
+    final Query<T> query = new Query<T>(fakeRecord);
+    T getRecord = driver.get(clazz, query);
+    assertNull(getRecord);
+
+    // Test fetch multiple objects does not exist returns empty list
+    assertEquals(driver.getMultiple(clazz, query).size(), 0);
+  }
+
+  public <T extends BaseRecord> void testPut(
+      StateStoreDriver driver, Class<T> clazz)
+          throws IllegalArgumentException, ReflectiveOperationException,
+          IOException, SecurityException {
+
+    driver.removeAll(clazz);
+    QueryResult<T> records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(clazz);
+      insertList.add(newRecord);
+    }
+
+    // Verify
+    assertTrue(driver.putAll(insertList, false, true));
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 10);
+
+    // Generate a new record with the same PK fields as an existing record
+    BaseRecord updatedRecord = generateFakeRecord(clazz);
+    BaseRecord existingRecord = records.getRecords().get(0);
+    Map<String, String> primaryKeys = existingRecord.getPrimaryKeys();
+    for (Entry<String, String> entry : primaryKeys.entrySet()) {
+      String key = entry.getKey();
+      String value = entry.getValue();
+      Class<?> fieldType = getFieldType(existingRecord, key);
+      Object field = fromString(value, fieldType);
+      assertTrue(setField(updatedRecord, key, field));
+    }
+
+    // Attempt an update of an existing entry, but it is not allowed.
+    assertFalse(driver.put(updatedRecord, false, true));
+
+    // Verify no update occurred, all original records are unchanged
+    QueryResult<T> newRecords = driver.get(clazz);
+    assertTrue(newRecords.getRecords().size() == 10);
+    assertEquals("A single entry was improperly updated in the store", 10,
+        countMatchingEntries(records.getRecords(), newRecords.getRecords()));
+
+    // Update the entry (allowing updates)
+    assertTrue(driver.put(updatedRecord, true, false));
+
+    // Verify that one entry no longer matches the original set
+    newRecords = driver.get(clazz);
+    assertEquals(10, newRecords.getRecords().size());
+    assertEquals(
+        "Record of type " + clazz + " not updated in the store", 9,
+        countMatchingEntries(records.getRecords(), newRecords.getRecords()));
+  }
+
+  private int countMatchingEntries(
+      Collection<? extends BaseRecord> committedList,
+      Collection<? extends BaseRecord> matchList) {
+
+    int matchingCount = 0;
+    for (BaseRecord committed : committedList) {
+      for (BaseRecord match : matchList) {
+        try {
+          if (match.getPrimaryKey().equals(committed.getPrimaryKey())) {
+            if (validateRecord(match, committed, false)) {
+              matchingCount++;
+            }
+            break;
+          }
+        } catch (Exception ex) {
+        }
+      }
+    }
+    return matchingCount;
+  }
+
+  public <T extends BaseRecord> void testRemove(
+      StateStoreDriver driver, Class<T> clazz)
+          throws IllegalArgumentException, IllegalAccessException, IOException {
+
+    // Remove all
+    assertTrue(driver.removeAll(clazz));
+    QueryResult<T> records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+
+    // Insert multiple
+    List<T> insertList = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      T newRecord = generateFakeRecord(clazz);
+      insertList.add(newRecord);
+    }
+
+    // Verify
+    assertTrue(driver.putAll(insertList, false, true));
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 10);
+
+    // Remove Single
+    assertTrue(driver.remove(records.getRecords().get(0)));
+
+    // Verify
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 9);
+
+    // Remove with filter
+    final T firstRecord = records.getRecords().get(0);
+    final Query<T> query0 = new Query<T>(firstRecord);
+    assertTrue(driver.remove(clazz, query0) > 0);
+
+    final T secondRecord = records.getRecords().get(1);
+    final Query<T> query1 = new Query<T>(secondRecord);
+    assertTrue(driver.remove(clazz, query1) > 0);
+
+    // Verify
+    records = driver.get(clazz);
+    assertEquals(records.getRecords().size(), 7);
+
+    // Remove all
+    assertTrue(driver.removeAll(clazz));
+
+    // Verify
+    records = driver.get(clazz);
+    assertTrue(records.getRecords().isEmpty());
+  }
+
+  public void testInsert(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  public void testPut(StateStoreDriver driver)
+      throws IllegalArgumentException, ReflectiveOperationException,
+      IOException, SecurityException {
+    // TODO add records
+  }
+
+  public void testRemove(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  public void testFetchErrors(StateStoreDriver driver)
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    // TODO add records
+  }
+
+  /**
+   * Sets the value of a field on the object.
+   *
+   * @param fieldName The string name of the field.
+   * @param data The data to pass to the field's setter.
+   *
+   * @return True if successful, fails if failed.
+   */
+  private static boolean setField(
+      BaseRecord record, String fieldName, Object data) {
+
+    Method m = locateSetter(record, fieldName);
+    if (m != null) {
+      try {
+        m.invoke(record, data);
+      } catch (Exception e) {
+        LOG.error("Cannot set field " + fieldName + " on object "
+            + record.getClass().getName() + " to data " + data + " of type "
+            + data.getClass(), e);
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Finds the appropriate setter for a field name.
+   *
+   * @param fieldName The legacy name of the field.
+   * @return The matching setter or null if not found.
+   */
+  private static Method locateSetter(BaseRecord record, String fieldName) {
+    for (Method m : record.getClass().getMethods()) {
+      if (m.getName().equalsIgnoreCase("set" + fieldName)) {
+        return m;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Returns all serializable fields in the object.
+   *
+   * @return Map with the fields.
+   */
+  private static Map<String, Class<?>> getFields(BaseRecord record) {
+    Map<String, Class<?>> getters = new HashMap<>();
+    for (Method m : record.getClass().getDeclaredMethods()) {
+      if (m.getName().startsWith("get")) {
+        try {
+          Class<?> type = m.getReturnType();
+          char[] c = m.getName().substring(3).toCharArray();
+          c[0] = Character.toLowerCase(c[0]);
+          String key = new String(c);
+          getters.put(key, type);
+        } catch (Exception e) {
+          LOG.error("Cannot execute getter " + m.getName()
+              + " on object " + record);
+        }
+      }
+    }
+    return getters;
+  }
+
+  /**
+   * Get the type of a field.
+   *
+   * @param fieldName
+   * @return Field type
+   */
+  private static Class<?> getFieldType(BaseRecord record, String fieldName) {
+    Method m = locateGetter(record, fieldName);
+    return m.getReturnType();
+  }
+
+  /**
+   * Fetches the value for a field name.
+   *
+   * @param fieldName the legacy name of the field.
+   * @return The field data or null if not found.
+   */
+  private static Object getField(BaseRecord record, String fieldName) {
+    Object result = null;
+    Method m = locateGetter(record, fieldName);
+    if (m != null) {
+      try {
+        result = m.invoke(record);
+      } catch (Exception e) {
+        LOG.error("Cannot get field " + fieldName + " on object " + record);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Finds the appropriate getter for a field name.
+   *
+   * @param fieldName The legacy name of the field.
+   * @return The matching getter or null if not found.
+   */
+  private static Method locateGetter(BaseRecord record, String fieldName) {
+    for (Method m : record.getClass().getMethods()) {
+      if (m.getName().equalsIgnoreCase("get" + fieldName)) {
+        return m;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Expands a data object from the store into an record object. Default store
+   * data type is a String. Override if additional serialization is required.
+   *
+   * @param data Object containing the serialized data. Only string is
+   *          supported.
+   * @param clazz Target object class to hold the deserialized data.
+   * @return An instance of the target data object initialized with the
+   *         deserialized data.
+   */
+  @Deprecated
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  private static <T> T fromString(String data, Class<T> clazz) {
+
+    if (data.equals("null")) {
+      return null;
+    } else if (clazz == String.class) {
+      return (T) data;
+    } else if (clazz == Long.class || clazz == long.class) {
+      return (T) Long.valueOf(data);
+    } else if (clazz == Integer.class || clazz == int.class) {
+      return (T) Integer.valueOf(data);
+    } else if (clazz == Double.class || clazz == double.class) {
+      return (T) Double.valueOf(data);
+    } else if (clazz == Float.class || clazz == float.class) {
+      return (T) Float.valueOf(data);
+    } else if (clazz == Boolean.class || clazz == boolean.class) {
+      return (T) Boolean.valueOf(data);
+    } else if (clazz.isEnum()) {
+      return (T) Enum.valueOf((Class<Enum>) clazz, data);
+    }
+    return null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
new file mode 100644
index 0000000..920e280
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFile.java
@@ -0,0 +1,64 @@
+/**
+ * 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.store.driver;
+
+import static org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils.getStateStoreConfiguration;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the FileSystem (e.g., HDFS) implementation of the State Store driver.
+ */
+public class TestStateStoreFile extends TestStateStoreDriverBase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = getStateStoreConfiguration(StateStoreFileImpl.class);
+    getStateStore(conf);
+  }
+
+  @Before
+  public void startup() throws IOException {
+    removeAll(getStateStoreDriver());
+  }
+
+  @Test
+  public void testInsert()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testUpdate()
+      throws IllegalArgumentException, ReflectiveOperationException,
+      IOException, SecurityException {
+    testPut(getStateStoreDriver());
+  }
+
+  @Test
+  public void testDelete()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testRemove(getStateStoreDriver());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.java
new file mode 100644
index 0000000..da2e51d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreFileSystem.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.hdfs.server.federation.store.driver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.store.FederationStateStoreTestUtils;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileSystemImpl;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test the FileSystem (e.g., HDFS) implementation of the State Store driver.
+ */
+public class TestStateStoreFileSystem extends TestStateStoreDriverBase {
+
+  private static MiniDFSCluster dfsCluster;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = FederationStateStoreTestUtils
+        .getStateStoreConfiguration(StateStoreFileSystemImpl.class);
+    conf.set(StateStoreFileSystemImpl.FEDERATION_STORE_FS_PATH,
+        "/hdfs-federation/");
+
+    // Create HDFS cluster to back the state tore
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    builder.numDataNodes(1);
+    dfsCluster = builder.build();
+    dfsCluster.waitClusterUp();
+    getStateStore(conf);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() {
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+  }
+
+  @Before
+  public void startup() throws IOException {
+    removeAll(getStateStoreDriver());
+  }
+
+  @Test
+  public void testInsert()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testUpdate()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testDelete()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testInsert(getStateStoreDriver());
+  }
+
+  @Test
+  public void testFetchErrors()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    testFetchErrors(getStateStoreDriver());
+  }
+}
\ 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


[09/50] [abbrv] hadoop git commit: YARN-6768. Improve performance of yarn api record toString and fromString. Contributed by Jonathan Eagles

Posted by in...@apache.org.
YARN-6768. Improve performance of yarn api record toString and fromString. Contributed by Jonathan Eagles


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

Branch: refs/heads/HDFS-10467
Commit: 24853bf32a045b8f029fb136edca2af03836c8d5
Parents: 770cc46
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Mon Jul 24 10:19:20 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Mon Jul 24 10:19:20 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/util/FastNumberFormat.java    |  44 ++++++
 .../hadoop/util/TestFastNumberFormat.java       |  46 ++++++
 .../yarn/api/records/ApplicationAttemptId.java  |  77 +++++-----
 .../hadoop/yarn/api/records/ApplicationId.java  |  54 +++----
 .../hadoop/yarn/api/records/ContainerId.java    | 139 +++++++++----------
 .../hadoop/yarn/api/records/ReservationId.java  |  21 ++-
 6 files changed, 220 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FastNumberFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FastNumberFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FastNumberFormat.java
new file mode 100644
index 0000000..9a6b63f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FastNumberFormat.java
@@ -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.
+ */
+
+package org.apache.hadoop.util;
+
+/**
+ * Fast thread-safe version of NumberFormat
+ */
+public class FastNumberFormat {
+
+  public static StringBuilder format(StringBuilder sb, long value, int minimumDigits) {
+    if (value < 0) {
+      sb.append('-');
+      value = -value;
+    }
+
+    long tmp = value;
+    do {
+      tmp /= 10;
+    } while (--minimumDigits > 0 && tmp > 0);
+
+    for (int i = minimumDigits; i > 0; --i) {
+      sb.append('0');
+    }
+
+    sb.append(value);
+    return sb;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFastNumberFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFastNumberFormat.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFastNumberFormat.java
new file mode 100644
index 0000000..c8935dd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestFastNumberFormat.java
@@ -0,0 +1,46 @@
+/**
+ * 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.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.text.NumberFormat;
+
+/**
+ * Test for FastNumberFormat
+ */
+public class TestFastNumberFormat {
+  private final int MIN_DIGITS = 6;
+
+  @Test(timeout = 1000)
+  public void testLongWithPadding() throws Exception {
+    NumberFormat numberFormat = NumberFormat.getInstance();
+    numberFormat.setGroupingUsed(false);
+    numberFormat.setMinimumIntegerDigits(6);
+    long[] testLongs = {1, 23, 456, 7890, 12345, 678901, 2345689, 0, -0, -1,
+      -23, -456, -7890, -12345, -678901, -2345689};
+    for (long l: testLongs) {
+      StringBuilder sb = new StringBuilder();
+      FastNumberFormat.format(sb, l, MIN_DIGITS);
+      String fastNumberStr = sb.toString();
+      Assert.assertEquals("Number formats should be equal",
+          numberFormat.format(l), fastNumberStr);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
index 5f3a68e..dbaacd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
@@ -18,18 +18,13 @@
 
 package org.apache.hadoop.yarn.api.records;
 
-import java.text.NumberFormat;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.FastNumberFormat;
 import org.apache.hadoop.yarn.util.Records;
 
-import com.google.common.base.Splitter;
-
 /**
  * <p><code>ApplicationAttemptId</code> denotes the particular <em>attempt</em>
  * of an <code>ApplicationMaster</code> for a given {@link ApplicationId}.</p>
@@ -42,12 +37,14 @@ import com.google.common.base.Splitter;
 @Stable
 public abstract class ApplicationAttemptId implements
     Comparable<ApplicationAttemptId> {
-  private static Splitter _spliter = Splitter.on('_').trimResults();
 
   @Private
   @Unstable
   public static final String appAttemptIdStrPrefix = "appattempt";
 
+  private static final String APP_ATTEMPT_ID_PREFIX = appAttemptIdStrPrefix
+    + '_';
+
   @Public
   @Unstable
   public static ApplicationAttemptId newInstance(ApplicationId appId,
@@ -84,16 +81,8 @@ public abstract class ApplicationAttemptId implements
   @Unstable
   protected abstract void setAttemptId(int attemptId);
 
-  static final ThreadLocal<NumberFormat> attemptIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
+  private static final int ATTEMPT_ID_MIN_DIGITS = 6;
+  private static final int APP_ID_MIN_DIGITS = 4;
 
   @Override
   public int hashCode() {
@@ -135,12 +124,14 @@ public abstract class ApplicationAttemptId implements
 
   @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder(appAttemptIdStrPrefix);
-    sb.append("_");
-    sb.append(this.getApplicationId().getClusterTimestamp()).append("_");
-    sb.append(ApplicationId.appIdFormat.get().format(
-        this.getApplicationId().getId()));
-    sb.append("_").append(attemptIdFormat.get().format(getAttemptId()));
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(APP_ATTEMPT_ID_PREFIX);
+    ApplicationId appId = getApplicationId();
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    FastNumberFormat.format(sb, appId.getId(), APP_ID_MIN_DIGITS);
+    sb.append('_');
+    FastNumberFormat.format(sb, getAttemptId(), ATTEMPT_ID_MIN_DIGITS);
     return sb.toString();
   }
 
@@ -148,29 +139,33 @@ public abstract class ApplicationAttemptId implements
   
   @Public
   @Stable
-  public static ApplicationAttemptId fromString(String applicationAttemptIdStr) {
-    Iterator<String> it = _spliter.split(applicationAttemptIdStr).iterator();
-    if (!it.next().equals(appAttemptIdStrPrefix)) {
+  public static ApplicationAttemptId fromString(String appAttemptIdStr) {
+    if (!appAttemptIdStr.startsWith(APP_ATTEMPT_ID_PREFIX)) {
       throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
-          + applicationAttemptIdStr);
+          + appAttemptIdStr);
     }
     try {
-      return toApplicationAttemptId(it);
+      int pos1 = APP_ATTEMPT_ID_PREFIX.length() - 1;
+      int pos2 = appAttemptIdStr.indexOf('_', pos1 + 1);
+      if (pos2 < 0) {
+        throw new IllegalArgumentException("Invalid AppAttemptId: "
+            + appAttemptIdStr);
+      }
+      long rmId = Long.parseLong(appAttemptIdStr.substring(pos1 + 1, pos2));
+      int pos3 = appAttemptIdStr.indexOf('_', pos2 + 1);
+      if (pos3 < 0) {
+        throw new IllegalArgumentException("Invalid AppAttemptId: "
+            + appAttemptIdStr);
+      }
+      int appId = Integer.parseInt(appAttemptIdStr.substring(pos2 + 1, pos3));
+      ApplicationId applicationId = ApplicationId.newInstance(rmId, appId);
+      int attemptId = Integer.parseInt(appAttemptIdStr.substring(pos3 + 1));
+      ApplicationAttemptId applicationAttemptId =
+        ApplicationAttemptId.newInstance(applicationId, attemptId);
+      return applicationAttemptId;
     } catch (NumberFormatException n) {
       throw new IllegalArgumentException("Invalid AppAttemptId: "
-          + applicationAttemptIdStr, n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid AppAttemptId: "
-          + applicationAttemptIdStr, e);
+          + appAttemptIdStr, n);
     }
   }
-  
-  private static ApplicationAttemptId toApplicationAttemptId(
-      Iterator<String> it) throws NumberFormatException {
-    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
-        Integer.parseInt(it.next()));
-    ApplicationAttemptId appAttemptId =
-        ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next()));
-    return appAttemptId;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
index 03a77ce..392e0a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
@@ -18,18 +18,13 @@
 
 package org.apache.hadoop.yarn.api.records;
 
-import java.text.NumberFormat;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.FastNumberFormat;
 import org.apache.hadoop.yarn.util.Records;
 
-import com.google.common.base.Splitter;
-
 /**
  * <p><code>ApplicationId</code> represents the <em>globally unique</em> 
  * identifier for an application.</p>
@@ -42,12 +37,13 @@ import com.google.common.base.Splitter;
 @Public
 @Stable
 public abstract class ApplicationId implements Comparable<ApplicationId> {
-  private static Splitter _spliter = Splitter.on('_').trimResults();
 
   @Private
   @Unstable
   public static final String appIdStrPrefix = "application";
 
+  private static final String APPLICATION_ID_PREFIX = appIdStrPrefix + '_';
+
   @Public
   @Unstable
   public static ApplicationId newInstance(long clusterTimestamp, int id) {
@@ -87,16 +83,7 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
 
   protected abstract void build();
   
-  static final ThreadLocal<NumberFormat> appIdFormat =
-    new ThreadLocal<NumberFormat>() {
-      @Override
-      public NumberFormat initialValue() {
-        NumberFormat fmt = NumberFormat.getInstance();
-        fmt.setGroupingUsed(false);
-        fmt.setMinimumIntegerDigits(4);
-        return fmt;
-      }
-    };
+  private static final int APP_ID_MIN_DIGITS = 4;
 
   @Override
   public int compareTo(ApplicationId other) {
@@ -110,37 +97,38 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
 
   @Override
   public String toString() {
-    return appIdStrPrefix + "_" + this.getClusterTimestamp() + "_" + appIdFormat
-        .get().format(getId());
-  }
-  
-  private static ApplicationId toApplicationId(
-      Iterator<String> it) throws NumberFormatException {
-    ApplicationId appId = ApplicationId.newInstance(Long.parseLong(it.next()),
-        Integer.parseInt(it.next()));
-    return appId;
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(APPLICATION_ID_PREFIX);
+    sb.append(getClusterTimestamp());
+    sb.append('_');
+    FastNumberFormat.format(sb, getId(), APP_ID_MIN_DIGITS);
+    return sb.toString();
   }
   
   @Public
   @Stable
   public static ApplicationId fromString(String appIdStr) {
-    Iterator<String> it = _spliter.split((appIdStr)).iterator();
-    if (!it.next().equals(appIdStrPrefix)) {
+    if (!appIdStr.startsWith(APPLICATION_ID_PREFIX)) {
       throw new IllegalArgumentException("Invalid ApplicationId prefix: "
           + appIdStr + ". The valid ApplicationId should start with prefix "
           + appIdStrPrefix);
     }
     try {
-      return toApplicationId(it);
+      int pos1 = APPLICATION_ID_PREFIX.length() - 1;
+      int pos2 = appIdStr.indexOf('_', pos1 + 1);
+      if (pos2 < 0) {
+        throw new IllegalArgumentException("Invalid ApplicationId: "
+            + appIdStr);
+      }
+      long rmId = Long.parseLong(appIdStr.substring(pos1 + 1, pos2));
+      int appId = Integer.parseInt(appIdStr.substring(pos2 + 1));
+      ApplicationId applicationId = ApplicationId.newInstance(rmId, appId);
+      return applicationId;
     } catch (NumberFormatException n) {
       throw new IllegalArgumentException("Invalid ApplicationId: "
           + appIdStr, n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid ApplicationId: "
-          + appIdStr, e);
     }
   }
-
   @Override
   public int hashCode() {
     // Generated by eclipse.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
index feddeca..e506dc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
@@ -18,16 +18,11 @@
 
 package org.apache.hadoop.yarn.api.records;
 
-import com.google.common.base.Splitter;
-
-import java.text.NumberFormat;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.FastNumberFormat;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -38,8 +33,7 @@ import org.apache.hadoop.yarn.util.Records;
 @Stable
 public abstract class ContainerId implements Comparable<ContainerId>{
   public static final long CONTAINER_ID_BITMASK = 0xffffffffffL;
-  private static final Splitter _SPLITTER = Splitter.on('_').trimResults();
-  private static final String CONTAINER_PREFIX = "container";
+  private static final String CONTAINER_PREFIX = "container_";
   private static final String EPOCH_PREFIX = "e";
 
   @Public
@@ -115,29 +109,13 @@ public abstract class ContainerId implements Comparable<ContainerId>{
   protected abstract void setContainerId(long id);
  
   
-  // TODO: fail the app submission if attempts are more than 10 or something
-  private static final ThreadLocal<NumberFormat> appAttemptIdAndEpochFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(2);
-          return fmt;
-        }
-      };
-  // TODO: Why thread local?
-  // ^ NumberFormat instances are not threadsafe
-  private static final ThreadLocal<NumberFormat> containerIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
+  private static final int APP_ID_MIN_DIGITS = 4;
+
+  private static final int ATTEMPT_ID_MIN_DIGITS = 2;
+
+  private static final int EPOCH_MIN_DIGITS = 2;
+
+  private static final int CONTAINER_ID_MIN_DIGITS = 6;
 
   @Override
   public int hashCode() {
@@ -185,72 +163,85 @@ public abstract class ContainerId implements Comparable<ContainerId>{
    */
   @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(CONTAINER_PREFIX + "_");
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(CONTAINER_PREFIX);
     long epoch = getContainerId() >> 40;
     if (epoch > 0) {
-      sb.append(EPOCH_PREFIX)
-          .append(appAttemptIdAndEpochFormat.get().format(epoch)).append("_");;
+      sb.append(EPOCH_PREFIX);
+      FastNumberFormat.format(sb, epoch, EPOCH_MIN_DIGITS);
+      sb.append('_');
     }
     ApplicationId appId = getApplicationAttemptId().getApplicationId();
-    sb.append(appId.getClusterTimestamp()).append("_");
-    sb.append(ApplicationId.appIdFormat.get().format(appId.getId()))
-        .append("_");
-    sb.append(
-        appAttemptIdAndEpochFormat.get().format(
-            getApplicationAttemptId().getAttemptId())).append("_");
-    sb.append(containerIdFormat.get()
-        .format(CONTAINER_ID_BITMASK & getContainerId()));
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    FastNumberFormat.format(sb, appId.getId(), APP_ID_MIN_DIGITS);
+    sb.append('_');
+    FastNumberFormat.format(sb, getApplicationAttemptId().getAttemptId(),
+        ATTEMPT_ID_MIN_DIGITS);
+    sb.append('_');
+    FastNumberFormat.format(sb, CONTAINER_ID_BITMASK & getContainerId(),
+        CONTAINER_ID_MIN_DIGITS);
     return sb.toString();
   }
 
   @Public
   @Stable
   public static ContainerId fromString(String containerIdStr) {
-    Iterator<String> it = _SPLITTER.split(containerIdStr).iterator();
-    if (!it.next().equals(CONTAINER_PREFIX)) {
+    if (!containerIdStr.startsWith(CONTAINER_PREFIX)) {
       throw new IllegalArgumentException("Invalid ContainerId prefix: "
           + containerIdStr);
     }
     try {
-      String epochOrClusterTimestampStr = it.next();
+      int pos1 = CONTAINER_PREFIX.length() - 1;
+
       long epoch = 0;
-      ApplicationAttemptId appAttemptID = null;
-      if (epochOrClusterTimestampStr.startsWith(EPOCH_PREFIX)) {
-        String epochStr = epochOrClusterTimestampStr;
-        epoch = Integer.parseInt(epochStr.substring(EPOCH_PREFIX.length()));
-        appAttemptID = toApplicationAttemptId(it);
-      } else {
-        String clusterTimestampStr = epochOrClusterTimestampStr;
-        long clusterTimestamp = Long.parseLong(clusterTimestampStr);
-        appAttemptID = toApplicationAttemptId(clusterTimestamp, it);
+      if (containerIdStr.regionMatches(pos1 + 1, EPOCH_PREFIX, 0,
+            EPOCH_PREFIX.length())) {
+        int pos2 = containerIdStr.indexOf('_', pos1 + 1);
+        if (pos2 < 0) {
+          throw new IllegalArgumentException("Invalid ContainerId: "
+              + containerIdStr);
+        }
+        String epochStr = containerIdStr.substring(
+            pos1 + 1 + EPOCH_PREFIX.length(), pos2);
+        epoch = Integer.parseInt(epochStr);
+        // rewind the current position
+        pos1 = pos2;
       }
-      long id = Long.parseLong(it.next());
+      int pos2 = containerIdStr.indexOf('_', pos1 + 1);
+      if (pos2 < 0) {
+        throw new IllegalArgumentException("Invalid ContainerId: "
+            + containerIdStr);
+      }
+      long clusterTimestamp = Long.parseLong(
+        containerIdStr.substring(pos1 + 1, pos2));
+
+      int pos3 = containerIdStr.indexOf('_', pos2 + 1);
+      if (pos3 < 0) {
+        throw new IllegalArgumentException("Invalid ContainerId: "
+            + containerIdStr);
+      }
+      int appId = Integer.parseInt(containerIdStr.substring(pos2 + 1, pos3));
+      ApplicationId applicationId = ApplicationId.newInstance(clusterTimestamp,
+          appId);
+      int pos4 = containerIdStr.indexOf('_', pos3 + 1);
+      if (pos4 < 0) {
+        throw new IllegalArgumentException("Invalid ContainerId: "
+            + containerIdStr);
+      }
+      int attemptId = Integer.parseInt(
+          containerIdStr.substring(pos3 + 1, pos4));
+      ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(applicationId, attemptId);
+      long id = Long.parseLong(containerIdStr.substring(pos4 + 1));
       long cid = (epoch << 40) | id;
-      ContainerId containerId = ContainerId.newContainerId(appAttemptID, cid);
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId, cid);
       return containerId;
     } catch (NumberFormatException n) {
       throw new IllegalArgumentException("Invalid ContainerId: "
           + containerIdStr, n);
-    } catch (NoSuchElementException e) {
-      throw new IllegalArgumentException("Invalid ContainerId: "
-          + containerIdStr, e);
     }
   }
 
-  private static ApplicationAttemptId toApplicationAttemptId(
-      Iterator<String> it) throws NumberFormatException {
-    return toApplicationAttemptId(Long.parseLong(it.next()), it);
-  }
-
-  private static ApplicationAttemptId toApplicationAttemptId(
-      long clusterTimestamp, Iterator<String> it) throws NumberFormatException {
-    ApplicationId appId = ApplicationId.newInstance(clusterTimestamp,
-        Integer.parseInt(it.next()));
-    ApplicationAttemptId appAttemptId =
-        ApplicationAttemptId.newInstance(appId, Integer.parseInt(it.next()));
-    return appAttemptId;
-  }
-
   protected abstract void build();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24853bf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
index 71474b1..a0c209d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationId.java
@@ -19,11 +19,11 @@
 package org.apache.hadoop.yarn.api.records;
 
 import java.io.IOException;
-import java.text.NumberFormat;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.FastNumberFormat;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -89,16 +89,7 @@ public abstract class ReservationId implements Comparable<ReservationId> {
 
   protected abstract void build();
 
-  static final ThreadLocal<NumberFormat> reservIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(4);
-          return fmt;
-        }
-      };
+  private static final int RESERVATION_ID_MIN_DIGITS = 4;
 
   @Override
   public int compareTo(ReservationId other) {
@@ -112,8 +103,12 @@ public abstract class ReservationId implements Comparable<ReservationId> {
 
   @Override
   public String toString() {
-    return reserveIdStrPrefix + this.getClusterTimestamp() + "_"
-        + reservIdFormat.get().format(getId());
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(reserveIdStrPrefix);
+    sb.append(getClusterTimestamp());
+    sb.append('_');
+    FastNumberFormat.format(sb, getId(), RESERVATION_ID_MIN_DIGITS);
+    return sb.toString();
   }
 
   /**


---------------------------------------------------------------------
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-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
new file mode 100644
index 0000000..3a32be1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -0,0 +1,856 @@
+/**
+ * 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 java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+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 java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A client proxy for Router -> NN communication using the NN ClientProtocol.
+ * <p>
+ * Provides routers to invoke remote ClientProtocol methods and handle
+ * retries/failover.
+ * <ul>
+ * <li>invokeSingle Make a single request to a single namespace
+ * <li>invokeSequential Make a sequential series of requests to multiple
+ * ordered namespaces until a condition is met.
+ * <li>invokeConcurrent Make concurrent requests to multiple namespaces and
+ * return all of the results.
+ * </ul>
+ * Also maintains a cached pool of connections to NNs. Connections are managed
+ * by the ConnectionManager and are unique to each user + NN. The size of the
+ * connection pool can be configured. Larger pools allow for more simultaneous
+ * requests to a single NN from a single user.
+ */
+public class RouterRpcClient {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterRpcClient.class);
+
+
+  /** Router identifier. */
+  private final String routerId;
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** Connection pool to the Namenodes per user for performance. */
+  private final ConnectionManager connectionManager;
+  /** Service to run asynchronous calls. */
+  private final ExecutorService executorService;
+  /** Retry policy for router -> NN communication. */
+  private final RetryPolicy retryPolicy;
+
+  /** Pattern to parse a stack trace line. */
+  private static final Pattern STACK_TRACE_PATTERN =
+      Pattern.compile("\\tat (.*)\\.(.*)\\((.*):(\\d*)\\)");
+
+
+  /**
+   * Create a router RPC client to manage remote procedure calls to NNs.
+   *
+   * @param conf Hdfs Configuation.
+   * @param resolver A NN resolver to determine the currently active NN in HA.
+   * @param monitor Optional performance monitor.
+   */
+  public RouterRpcClient(Configuration conf, String identifier,
+      ActiveNamenodeResolver resolver) {
+
+    this.routerId = identifier;
+
+    this.namenodeResolver = resolver;
+
+    this.connectionManager = new ConnectionManager(conf);
+    this.connectionManager.start();
+
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat("RPC Router Client-%d")
+        .build();
+    this.executorService = Executors.newCachedThreadPool(threadFactory);
+
+    int maxFailoverAttempts = conf.getInt(
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
+    int maxRetryAttempts = conf.getInt(
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
+        HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+    int failoverSleepBaseMillis = conf.getInt(
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
+    int failoverSleepMaxMillis = conf.getInt(
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+        HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
+    this.retryPolicy = RetryPolicies.failoverOnNetworkException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts, maxRetryAttempts,
+        failoverSleepBaseMillis, failoverSleepMaxMillis);
+  }
+
+  /**
+   * Shutdown the client.
+   */
+  public void shutdown() {
+    if (this.connectionManager != null) {
+      this.connectionManager.close();
+    }
+    if (this.executorService != null) {
+      this.executorService.shutdownNow();
+    }
+  }
+
+  /**
+   * Total number of available sockets between the router and NNs.
+   *
+   * @return Number of namenode clients.
+   */
+  public int getNumConnections() {
+    return this.connectionManager.getNumConnections();
+  }
+
+  /**
+   * Total number of available sockets between the router and NNs.
+   *
+   * @return Number of namenode clients.
+   */
+  public int getNumActiveConnections() {
+    return this.connectionManager.getNumActiveConnections();
+  }
+
+  /**
+   * Total number of open connection pools to a NN. Each connection pool.
+   * represents one user + one NN.
+   *
+   * @return Number of connection pools.
+   */
+  public int getNumConnectionPools() {
+    return this.connectionManager.getNumConnectionPools();
+  }
+
+  /**
+   * Number of connections between the router and NNs being created sockets.
+   *
+   * @return Number of connections waiting to be created.
+   */
+  public int getNumCreatingConnections() {
+    return this.connectionManager.getNumCreatingConnections();
+  }
+
+  /**
+   * Get ClientProtocol proxy client for a NameNode. Each combination of user +
+   * NN must use a unique proxy client. Previously created clients are cached
+   * and stored in a connection pool by the ConnectionManager.
+   *
+   * @param ugi User group information.
+   * @param nsId Nameservice identifier.
+   * @param rpcAddress ClientProtocol RPC server address of the NN.
+   * @return ConnectionContext containing a ClientProtocol proxy client for the
+   *         NN + current user.
+   * @throws IOException If we cannot get a connection to the NameNode.
+   */
+  private ConnectionContext getConnection(
+      UserGroupInformation ugi, String nsId, String rpcAddress)
+          throws IOException {
+    ConnectionContext connection = null;
+    try {
+      // Each proxy holds the UGI info for the current user when it is created.
+      // This cache does not scale very well, one entry per user per namenode,
+      // and may need to be adjusted and/or selectively pruned. The cache is
+      // important due to the excessive overhead of creating a new proxy wrapper
+      // for each individual request.
+
+      // TODO Add tokens from the federated UGI
+      connection = this.connectionManager.getConnection(ugi, rpcAddress);
+      LOG.debug("User {} NN {} is using connection {}",
+          ugi.getUserName(), rpcAddress, connection);
+    } catch (Exception ex) {
+      LOG.error("Cannot open NN client to address: {}", rpcAddress, ex);
+    }
+
+    if (connection == null) {
+      throw new IOException("Cannot get a connection to " + rpcAddress);
+    }
+    return connection;
+  }
+
+  /**
+   * Convert an exception to an IOException.
+   *
+   * For a non-IOException, wrap it with IOException. For a RemoteException,
+   * unwrap it. For an IOException which is not a RemoteException, return it.
+   *
+   * @param e Exception to convert into an exception.
+   * @return Created IO exception.
+   */
+  private static IOException toIOException(Exception e) {
+    if (e instanceof RemoteException) {
+      return ((RemoteException) e).unwrapRemoteException();
+    }
+    if (e instanceof IOException) {
+      return (IOException)e;
+    }
+    return new IOException(e);
+  }
+
+  /**
+   * If we should retry the RPC call.
+   *
+   * @param ex Exception reported.
+   * @param retryCount Number of retries.
+   * @return Retry decision.
+   * @throws IOException Original exception if the retry policy generates one.
+   */
+  private RetryDecision shouldRetry(final IOException ioe, final int retryCount)
+      throws IOException {
+    try {
+      final RetryPolicy.RetryAction a =
+          this.retryPolicy.shouldRetry(ioe, retryCount, 0, true);
+      return a.action;
+    } catch (Exception ex) {
+      LOG.error("Re-throwing API exception, no more retries", ex);
+      throw toIOException(ex);
+    }
+  }
+
+  /**
+   * Invokes a method against the ClientProtocol proxy server. If a standby
+   * exception is generated by the call to the client, retries using the
+   * alternate server.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param ugi User group information.
+   * @param namenodes A prioritized list of namenodes within the same
+   *                  nameservice.
+   * @param method Remote ClientProtcol method to invoke.
+   * @param params Variable list of parameters matching the method.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  private Object invokeMethod(
+      final UserGroupInformation ugi,
+      final List<? extends FederationNamenodeContext> namenodes,
+      final Method method, final Object... params) throws IOException {
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("No namenodes to invoke " + method.getName() +
+          " with params " + Arrays.toString(params) + " from " + this.routerId);
+    }
+
+    Object ret = null;
+    boolean failover = false;
+    Map<FederationNamenodeContext, IOException> ioes = new LinkedHashMap<>();
+    for (FederationNamenodeContext namenode : namenodes) {
+      ConnectionContext connection = null;
+      try {
+        String nsId = namenode.getNameserviceId();
+        String rpcAddress = namenode.getRpcAddress();
+        connection = this.getConnection(ugi, nsId, rpcAddress);
+        ProxyAndInfo<ClientProtocol> client = connection.getClient();
+        ClientProtocol proxy = client.getProxy();
+        ret = invoke(0, method, proxy, params);
+        if (failover) {
+          // Success on alternate server, update
+          InetSocketAddress address = client.getAddress();
+          namenodeResolver.updateActiveNamenode(nsId, address);
+        }
+        return ret;
+      } catch (IOException ioe) {
+        ioes.put(namenode, ioe);
+        if (ioe instanceof StandbyException) {
+          // Fail over indicated by retry policy and/or NN
+          failover = true;
+        } else if (ioe instanceof RemoteException) {
+          // RemoteException returned by NN
+          throw (RemoteException) ioe;
+        } else {
+          // Other communication error, this is a failure
+          // Communication retries are handled by the retry policy
+          throw ioe;
+        }
+      } finally {
+        if (connection != null) {
+          connection.release();
+        }
+      }
+    }
+
+    // All namenodes were unavailable or in standby
+    String msg = "No namenode available to invoke " + method.getName() + " " +
+        Arrays.toString(params);
+    LOG.error(msg);
+    for (Entry<FederationNamenodeContext, IOException> entry :
+        ioes.entrySet()) {
+      FederationNamenodeContext namenode = entry.getKey();
+      String nsId = namenode.getNameserviceId();
+      String nnId = namenode.getNamenodeId();
+      String addr = namenode.getRpcAddress();
+      IOException ioe = entry.getValue();
+      if (ioe instanceof StandbyException) {
+        LOG.error("{} {} at {} is in Standby", nsId, nnId, addr);
+      } else {
+        LOG.error("{} {} at {} error: \"{}\"",
+            nsId, nnId, addr, ioe.getMessage());
+      }
+    }
+    throw new StandbyException(msg);
+  }
+
+  /**
+   * Invokes a method on the designated object. Catches exceptions specific to
+   * the invocation.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param method Method to invoke
+   * @param obj Target object for the method
+   * @param params Variable parameters
+   * @return Response from the remote server
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private Object invoke(int retryCount, final Method method, final Object obj,
+      final Object... params) throws IOException {
+    try {
+      return method.invoke(obj, params);
+    } catch (IllegalAccessException e) {
+      LOG.error("Unexpected exception while proxying API", e);
+      return null;
+    } catch (IllegalArgumentException e) {
+      LOG.error("Unexpected exception while proxying API", e);
+      return null;
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        IOException ioe = (IOException) cause;
+        // Check if we should retry.
+        RetryDecision decision = shouldRetry(ioe, retryCount);
+        if (decision == RetryDecision.RETRY) {
+          // retry
+          return invoke(++retryCount, method, obj, params);
+        } else if (decision == RetryDecision.FAILOVER_AND_RETRY) {
+          // failover, invoker looks for standby exceptions for failover.
+          if (ioe instanceof StandbyException) {
+            throw ioe;
+          } else {
+            throw new StandbyException(ioe.getMessage());
+          }
+        } else {
+          if (ioe instanceof RemoteException) {
+            RemoteException re = (RemoteException) ioe;
+            ioe = re.unwrapRemoteException();
+            ioe = getCleanException(ioe);
+          }
+          throw ioe;
+        }
+      } else {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  /**
+   * Get a clean copy of the exception. Sometimes the exceptions returned by the
+   * server contain the full stack trace in the message.
+   *
+   * @param ioe Exception to clean up.
+   * @return Copy of the original exception with a clean message.
+   */
+  private static IOException getCleanException(IOException ioe) {
+    IOException ret = null;
+
+    String msg = ioe.getMessage();
+    Throwable cause = ioe.getCause();
+    StackTraceElement[] stackTrace = ioe.getStackTrace();
+
+    // Clean the message by removing the stack trace
+    int index = msg.indexOf("\n");
+    if (index > 0) {
+      String[] msgSplit = msg.split("\n");
+      msg = msgSplit[0];
+
+      // Parse stack trace from the message
+      List<StackTraceElement> elements = new LinkedList<>();
+      for (int i=1; i<msgSplit.length; i++) {
+        String line = msgSplit[i];
+        Matcher matcher = STACK_TRACE_PATTERN.matcher(line);
+        if (matcher.find()) {
+          String declaringClass = matcher.group(1);
+          String methodName = matcher.group(2);
+          String fileName = matcher.group(3);
+          int lineNumber = Integer.parseInt(matcher.group(4));
+          StackTraceElement element = new StackTraceElement(
+              declaringClass, methodName, fileName, lineNumber);
+          elements.add(element);
+        }
+      }
+      stackTrace = elements.toArray(new StackTraceElement[elements.size()]);
+    }
+
+    // Create the new output exception
+    if (ioe instanceof RemoteException) {
+      RemoteException re = (RemoteException)ioe;
+      ret = new RemoteException(re.getClassName(), msg);
+    } else {
+      // Try the simple constructor and initialize the fields
+      Class<? extends IOException> ioeClass = ioe.getClass();
+      try {
+        Constructor<? extends IOException> constructor =
+            ioeClass.getDeclaredConstructor(String.class);
+        ret = constructor.newInstance(msg);
+      } catch (ReflectiveOperationException e) {
+        // If there are errors, just use the input one
+        LOG.error("Could not create exception {}", ioeClass.getSimpleName(), e);
+        ret = ioe;
+      }
+    }
+    if (ret != null) {
+      ret.initCause(cause);
+      ret.setStackTrace(stackTrace);
+    }
+
+    return ret;
+  }
+
+  /**
+   * Invokes a ClientProtocol method. Determines the target nameservice via a
+   * provided block.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param block Block used to determine appropriate nameservice.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingle(final ExtendedBlock block, RemoteMethod method)
+      throws IOException {
+    String bpId = block.getBlockPoolId();
+    return invokeSingleBlockPool(bpId, method);
+  }
+
+  /**
+   * Invokes a ClientProtocol method. Determines the target nameservice using
+   * the block pool id.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param bpId Block pool identifier.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingleBlockPool(final String bpId, RemoteMethod method)
+      throws IOException {
+    String nsId = getNameserviceForBlockPoolId(bpId);
+    return invokeSingle(nsId, method);
+  }
+
+  /**
+   * Invokes a ClientProtocol method against the specified namespace.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param nsId Target namespace for the method.
+   * @param method The remote method and parameters to invoke.
+   * @return The result of invoking the method.
+   * @throws IOException
+   */
+  public Object invokeSingle(final String nsId, RemoteMethod method)
+      throws IOException {
+    UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    List<? extends FederationNamenodeContext> nns =
+        getNamenodesForNameservice(nsId);
+    RemoteLocationContext loc = new RemoteLocation(nsId, "/");
+    return invokeMethod(ugi, nns, method.getMethod(), method.getParams(loc));
+  }
+
+  /**
+   * Invokes a single proxy call for a single location.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param location RemoteLocation to invoke.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @return The result of invoking the method if successful.
+   * @throws IOException
+   */
+  public Object invokeSingle(final RemoteLocationContext location,
+      RemoteMethod remoteMethod) throws IOException {
+    List<RemoteLocationContext> locations = Collections.singletonList(location);
+    return invokeSequential(locations, remoteMethod);
+  }
+
+  /**
+   * Invokes sequential proxy calls to different locations. Continues to invoke
+   * calls until a call returns without throwing a remote exception.
+   *
+   * @param locations List of locations/nameservices to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @return The result of the first successful call, or if no calls are
+   *         successful, the result of the last RPC call executed.
+   * @throws IOException if the success condition is not met and one of the RPC
+   *           calls generated a remote exception.
+   */
+  public Object invokeSequential(
+      final List<? extends RemoteLocationContext> locations,
+      final RemoteMethod remoteMethod) throws IOException {
+    return invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  /**
+   * Invokes sequential proxy calls to different locations. Continues to invoke
+   * calls until the success condition is met, or until all locations have been
+   * attempted.
+   *
+   * The success condition may be specified by:
+   * <ul>
+   * <li>An expected result class
+   * <li>An expected result value
+   * </ul>
+   *
+   * If no expected result class/values are specified, the success condition is
+   * a call that does not throw a remote exception.
+   *
+   * @param locations List of locations/nameservices to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @param expectedResultClass In order to be considered a positive result, the
+   *          return type must be of this class.
+   * @param expectedResultValue In order to be considered a positive result, the
+   *          return value must equal the value of this object.
+   * @return The result of the first successful call, or if no calls are
+   *         successful, the result of the first RPC call executed.
+   * @throws IOException if the success condition is not met, return the first
+   *                     remote exception generated.
+   */
+  public Object invokeSequential(
+      final List<? extends RemoteLocationContext> locations,
+      final RemoteMethod remoteMethod, Class<?> expectedResultClass,
+      Object expectedResultValue) throws IOException {
+
+    final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    final Method m = remoteMethod.getMethod();
+    IOException firstThrownException = null;
+    IOException lastThrownException = null;
+    Object firstResult = null;
+    // Invoke in priority order
+    for (final RemoteLocationContext loc : locations) {
+      String ns = loc.getNameserviceId();
+      List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(ns);
+      try {
+        Object[] params = remoteMethod.getParams(loc);
+        Object result = invokeMethod(ugi, namenodes, m, params);
+        // Check if the result is what we expected
+        if (isExpectedClass(expectedResultClass, result) &&
+            isExpectedValue(expectedResultValue, result)) {
+          // Valid result, stop here
+          return result;
+        }
+        if (firstResult == null) {
+          firstResult = result;
+        }
+      } catch (IOException ioe) {
+        // Record it and move on
+        lastThrownException = (IOException) ioe;
+        if (firstThrownException == null) {
+          firstThrownException = lastThrownException;
+        }
+      } catch (Exception e) {
+        // Unusual error, ClientProtocol calls always use IOException (or
+        // RemoteException). Re-wrap in IOException for compatibility with
+        // ClientProtcol.
+        LOG.error("Unexpected exception {} proxying {} to {}",
+            e.getClass(), m.getName(), ns, e);
+        lastThrownException = new IOException(
+            "Unexpected exception proxying API " + e.getMessage(), e);
+        if (firstThrownException == null) {
+          firstThrownException = lastThrownException;
+        }
+      }
+    }
+
+    if (firstThrownException != null) {
+      // re-throw the last exception thrown for compatibility
+      throw firstThrownException;
+    }
+    // Return the last result, whether it is the value we are looking for or a
+    return firstResult;
+  }
+
+  /**
+   * Checks if a result matches the required result class.
+   *
+   * @param expectedResultClass Required result class, null to skip the check.
+   * @param result The result to check.
+   * @return True if the result is an instance of the required class or if the
+   *         expected class is null.
+   */
+  private static boolean isExpectedClass(Class<?> expectedClass, Object clazz) {
+    if (expectedClass == null) {
+      return true;
+    } else if (clazz == null) {
+      return false;
+    } else {
+      return expectedClass.isInstance(clazz);
+    }
+  }
+
+  /**
+   * Checks if a result matches the expected value.
+   *
+   * @param expectedResultValue The expected value, null to skip the check.
+   * @param result The result to check.
+   * @return True if the result is equals to the expected value or if the
+   *         expected value is null.
+   */
+  private static boolean isExpectedValue(Object expectedValue, Object value) {
+    if (expectedValue == null) {
+      return true;
+    } else if (value == null) {
+      return false;
+    } else {
+      return value.equals(expectedValue);
+    }
+  }
+
+  /**
+   * Invokes multiple concurrent proxy calls to different clients. Returns an
+   * array of results.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param locations List of remote locations to call concurrently.
+   * @param remoteMethod The remote method and parameters to invoke.
+   * @param requireResponse If true an exception will be thrown if all calls do
+   *          not complete. If false exceptions are ignored and all data results
+   *          successfully received are returned.
+   * @param standby If the requests should go to the standby namenodes too.
+   * @return Result of invoking the method per subcluster: nsId -> result.
+   * @throws IOException If requiredResponse=true and any of the calls throw an
+   *           exception.
+   */
+  @SuppressWarnings("unchecked")
+  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+      final Collection<T> locations, final RemoteMethod method,
+      boolean requireResponse, boolean standby) throws IOException {
+
+    final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
+    final Method m = method.getMethod();
+
+    if (locations.size() == 1) {
+      // Shortcut, just one call
+      T location = locations.iterator().next();
+      String ns = location.getNameserviceId();
+      final List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(ns);
+      Object[] paramList = method.getParams(location);
+      Object result = invokeMethod(ugi, namenodes, m, paramList);
+      return Collections.singletonMap(location, result);
+    }
+
+    List<T> orderedLocations = new LinkedList<>();
+    Set<Callable<Object>> callables = new HashSet<>();
+    for (final T location : locations) {
+      String nsId = location.getNameserviceId();
+      final List<? extends FederationNamenodeContext> namenodes =
+          getNamenodesForNameservice(nsId);
+      final Object[] paramList = method.getParams(location);
+      if (standby) {
+        // Call the objectGetter to all NNs (including standby)
+        for (final FederationNamenodeContext nn : namenodes) {
+          String nnId = nn.getNamenodeId();
+          final List<FederationNamenodeContext> nnList =
+              Collections.singletonList(nn);
+          T nnLocation = location;
+          if (location instanceof RemoteLocation) {
+            nnLocation = (T)new RemoteLocation(nsId, nnId, location.getDest());
+          }
+          orderedLocations.add(nnLocation);
+          callables.add(new Callable<Object>() {
+            public Object call() throws Exception {
+              return invokeMethod(ugi, nnList, m, paramList);
+            }
+          });
+        }
+      } else {
+        // Call the objectGetter in order of nameservices in the NS list
+        orderedLocations.add(location);
+        callables.add(new Callable<Object>() {
+          public Object call() throws Exception {
+            return invokeMethod(ugi, namenodes, m, paramList);
+          }
+        });
+      }
+    }
+
+    try {
+      List<Future<Object>> futures = executorService.invokeAll(callables);
+      Map<T, Object> results = new TreeMap<>();
+      Map<T, IOException> exceptions = new TreeMap<>();
+      for (int i=0; i<futures.size(); i++) {
+        T location = orderedLocations.get(i);
+        try {
+          Future<Object> future = futures.get(i);
+          Object result = future.get();
+          results.put(location, result);
+        } catch (ExecutionException ex) {
+          Throwable cause = ex.getCause();
+          LOG.debug("Canot execute {} in {}: {}",
+              m.getName(), location, cause.getMessage());
+
+          // Convert into IOException if needed
+          IOException ioe = null;
+          if (cause instanceof IOException) {
+            ioe = (IOException) cause;
+          } else {
+            ioe = new IOException("Unhandled exception while proxying API " +
+                m.getName() + ": " + cause.getMessage(), cause);
+          }
+
+          // Response from all servers required, use this error.
+          if (requireResponse) {
+            throw ioe;
+          }
+
+          // Store the exceptions
+          exceptions.put(location, ioe);
+        }
+      }
+
+      // Throw the exception for the first location if there are no results
+      if (results.isEmpty()) {
+        T location = orderedLocations.get(0);
+        IOException ioe = exceptions.get(location);
+        if (ioe != null) {
+          throw ioe;
+        }
+      }
+
+      return results;
+    } catch (InterruptedException ex) {
+      LOG.error("Unexpected error while invoking API: {}", ex.getMessage());
+      throw new IOException(
+          "Unexpected error while invoking API " + ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Get a prioritized list of NNs that share the same nameservice ID (in the
+   * same namespace). NNs that are reported as ACTIVE will be first in the list.
+   *
+   * @param nameserviceId The nameservice ID for the namespace.
+   * @return A prioritized list of NNs to use for communication.
+   * @throws IOException If a NN cannot be located for the nameservice ID.
+   */
+  private List<? extends FederationNamenodeContext> getNamenodesForNameservice(
+      final String nsId) throws IOException {
+
+    final List<? extends FederationNamenodeContext> namenodes =
+        namenodeResolver.getNamenodesForNameserviceId(nsId);
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("Cannot locate a registered namenode for " + nsId +
+          " from " + this.routerId);
+    }
+    return namenodes;
+  }
+
+  /**
+   * Get a prioritized list of NNs that share the same block pool ID (in the
+   * same namespace). NNs that are reported as ACTIVE will be first in the list.
+   *
+   * @param blockPoolId The blockpool ID for the namespace.
+   * @return A prioritized list of NNs to use for communication.
+   * @throws IOException If a NN cannot be located for the block pool ID.
+   */
+  private List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      final String bpId) throws IOException {
+
+    List<? extends FederationNamenodeContext> namenodes =
+        namenodeResolver.getNamenodesForBlockPoolId(bpId);
+
+    if (namenodes == null || namenodes.isEmpty()) {
+      throw new IOException("Cannot locate a registered namenode for " + bpId +
+          " from " + this.routerId);
+    }
+    return namenodes;
+  }
+
+  /**
+   * Get the nameservice identifier for a block pool.
+   *
+   * @param bpId Identifier of the block pool.
+   * @return Nameservice identifier.
+   * @throws IOException If a NN cannot be located for the block pool ID.
+   */
+  private String getNameserviceForBlockPoolId(final String bpId)
+      throws IOException {
+    List<? extends FederationNamenodeContext> namenodes =
+        getNamenodesForBlockPoolId(bpId);
+    FederationNamenodeContext namenode = namenodes.get(0);
+    return namenode.getNameserviceId();
+  }
+}
\ 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


[36/50] [abbrv] hadoop git commit: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/Hamlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/Hamlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/Hamlet.java
new file mode 100644
index 0000000..05e1b79
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/Hamlet.java
@@ -0,0 +1,30557 @@
+/**
+* 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.
+*/
+
+// Generated by HamletGen. Do NOT edit!
+package org.apache.hadoop.yarn.webapp.hamlet2;
+import static java.util.EnumSet.of;
+import static org.apache.hadoop.yarn.webapp.hamlet2.HamletImpl.EOpt.ENDTAG;
+import static org.apache.hadoop.yarn.webapp.hamlet2.HamletImpl.EOpt.INLINE;
+import static org.apache.hadoop.yarn.webapp.hamlet2.HamletImpl.EOpt.PRE;
+
+import java.io.PrintWriter;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+public class Hamlet extends HamletImpl implements HamletSpec._Html {
+  public Hamlet(PrintWriter out, int nestLevel, boolean wasInline) {
+    super(out, nestLevel, wasInline);
+  }
+
+  static EnumSet<EOpt> opt(boolean endTag, boolean inline, boolean pre) {
+    EnumSet<EOpt> opts = of(ENDTAG);
+    if (!endTag) opts.remove(ENDTAG);
+    if (inline) opts.add(INLINE);
+    if (pre) opts.add(PRE);
+    return opts;
+  }
+
+  public class HTML<T extends __> extends EImp<T> implements HamletSpec.HTML {
+    public HTML(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public HTML<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public HTML<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public HEAD<HTML<T>> head() {
+      closeAttrs();
+      return head_(this, false);
+    }
+
+    @Override
+    public BODY<HTML<T>> body() {
+      closeAttrs();
+      return body_(this, false);
+    }
+
+    @Override
+    public BODY<HTML<T>> body(String selector) {
+      return setSelector(body(), selector);
+    }
+
+    @Override
+    public BASE<HTML<T>> base() {
+      closeAttrs();
+      return base_(this, false);
+    }
+
+    @Override
+    public HTML<T> base(String href) {
+      return base().$href(href).__();
+    }
+
+    @Override
+    public TITLE<HTML<T>> title() {
+      closeAttrs();
+      return title_(this, false);
+    }
+
+    @Override
+    public HTML<T> title(String cdata) {
+      return title().__(cdata).__();
+    }
+
+    @Override
+    public STYLE<HTML<T>> style() {
+      closeAttrs();
+      return style_(this, false);
+    }
+
+    @Override
+    public HTML<T> style(Object... lines) {
+      return style().$type("text/css").__(lines).__();
+    }
+
+    @Override
+    public LINK<HTML<T>> link() {
+      closeAttrs();
+      return link_(this, false);
+    }
+
+    @Override
+    public HTML<T> link(String href) {
+      return setLinkHref(link(), href).__();
+    }
+
+    @Override
+    public META<HTML<T>> meta() {
+      closeAttrs();
+      return meta_(this, false);
+    }
+
+    @Override
+    public HTML<T> meta(String name, String content) {
+      return meta().$name(name).$content(content).__();
+    }
+
+    @Override
+    public HTML<T> meta_http(String header, String content) {
+      return meta().$http_equiv(header).$content(content).__();
+    }
+
+    @Override
+    public SCRIPT<HTML<T>> script() {
+      closeAttrs();
+      return script_(this, false);
+    }
+
+    @Override
+    public HTML<T> script(String src) {
+      return setScriptSrc(script(), src).__();
+    }
+
+    @Override
+    public OBJECT<HTML<T>> object() {
+      closeAttrs();
+      return object_(this, true);
+    }
+
+    @Override
+    public OBJECT<HTML<T>> object(String selector) {
+      return setSelector(object(), selector);
+    }
+
+    @Override
+    public TABLE<HTML<T>> table() {
+      closeAttrs();
+      return table_(this, false);
+    }
+
+    @Override
+    public TABLE<HTML<T>> table(String selector) {
+      return setSelector(table(), selector);
+    }
+
+    @Override
+    public HTML<T> address(String cdata) {
+      return address().__(cdata).__();
+    }
+
+    @Override
+    public ADDRESS<HTML<T>> address() {
+      closeAttrs();
+      return address_(this, false);
+    }
+
+    @Override
+    public P<HTML<T>> p(String selector) {
+      return setSelector(p(), selector);
+    }
+
+    @Override
+    public P<HTML<T>> p() {
+      closeAttrs();
+      return p_(this, false);
+    }
+
+    @Override
+    public HTML<T> __(Class<? extends SubView> cls) {
+      _v(cls);
+      return this;
+    }
+
+    @Override
+    public HR<HTML<T>> hr() {
+      closeAttrs();
+      return hr_(this, false);
+    }
+
+    @Override
+    public HTML<T> hr(String selector) {
+      return setSelector(hr(), selector).__();
+    }
+
+    @Override
+    public DL<HTML<T>> dl(String selector) {
+      return setSelector(dl(), selector);
+    }
+
+    @Override
+    public DL<HTML<T>> dl() {
+      closeAttrs();
+      return dl_(this, false);
+    }
+
+    @Override
+    public DIV<HTML<T>> div(String selector) {
+      return setSelector(div(), selector);
+    }
+
+    @Override
+    public DIV<HTML<T>> div() {
+      closeAttrs();
+      return div_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<HTML<T>> blockquote() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<HTML<T>> bq() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public HTML<T> h1(String cdata) {
+      return h1().__(cdata).__();
+    }
+
+    @Override
+    public H1<HTML<T>> h1() {
+      closeAttrs();
+      return h1_(this, false);
+    }
+
+    @Override
+    public HTML<T> h1(String selector, String cdata) {
+      return setSelector(h1(), selector).__(cdata).__();
+    }
+
+    @Override
+    public HTML<T> h2(String cdata) {
+      return h2().__(cdata).__();
+    }
+
+    @Override
+    public H2<HTML<T>> h2() {
+      closeAttrs();
+      return h2_(this, false);
+    }
+
+    @Override
+    public HTML<T> h2(String selector, String cdata) {
+      return setSelector(h2(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H3<HTML<T>> h3() {
+      closeAttrs();
+      return h3_(this, false);
+    }
+
+    @Override
+    public HTML<T> h3(String cdata) {
+      return h3().__(cdata).__();
+    }
+
+    @Override
+    public HTML<T> h3(String selector, String cdata) {
+      return setSelector(h3(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H4<HTML<T>> h4() {
+      closeAttrs();
+      return h4_(this, false);
+    }
+
+    @Override
+    public HTML<T> h4(String cdata) {
+      return h4().__(cdata).__();
+    }
+
+    @Override
+    public HTML<T> h4(String selector, String cdata) {
+      return setSelector(h4(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H5<HTML<T>> h5() {
+      closeAttrs();
+      return h5_(this, false);
+    }
+
+    @Override
+    public HTML<T> h5(String cdata) {
+      return h5().__(cdata).__();
+    }
+
+    @Override
+    public HTML<T> h5(String selector, String cdata) {
+      return setSelector(h5(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H6<HTML<T>> h6() {
+      closeAttrs();
+      return h6_(this, false);
+    }
+
+    @Override
+    public HTML<T> h6(String cdata) {
+      return h6().__(cdata).__();
+    }
+
+    @Override
+    public HTML<T> h6(String selector, String cdata) {
+      return setSelector(h6(), selector).__(cdata).__();
+    }
+
+    @Override
+    public UL<HTML<T>> ul() {
+      closeAttrs();
+      return ul_(this, false);
+    }
+
+    @Override
+    public UL<HTML<T>> ul(String selector) {
+      return setSelector(ul(), selector);
+    }
+
+    @Override
+    public OL<HTML<T>> ol() {
+      closeAttrs();
+      return ol_(this, false);
+    }
+
+    @Override
+    public OL<HTML<T>> ol(String selector) {
+      return setSelector(ol(), selector);
+    }
+
+    @Override
+    public PRE<HTML<T>> pre() {
+      closeAttrs();
+      return pre_(this, false);
+    }
+
+    @Override
+    public PRE<HTML<T>> pre(String selector) {
+      return setSelector(pre(), selector);
+    }
+
+    @Override
+    public FORM<HTML<T>> form() {
+      closeAttrs();
+      return form_(this, false);
+    }
+
+    @Override
+    public FORM<HTML<T>> form(String selector) {
+      return setSelector(form(), selector);
+    }
+
+    @Override
+    public FIELDSET<HTML<T>> fieldset() {
+      closeAttrs();
+      return fieldset_(this, false);
+    }
+
+    @Override
+    public FIELDSET<HTML<T>> fieldset(String selector) {
+      return setSelector(fieldset(), selector);
+    }
+
+    @Override
+    public INS<HTML<T>> ins() {
+      closeAttrs();
+      return ins_(this, true);
+    }
+
+    @Override
+    public HTML<T> ins(String cdata) {
+      return ins().__(cdata).__();
+    }
+
+    @Override
+    public DEL<HTML<T>> del() {
+      closeAttrs();
+      return del_(this, true);
+    }
+
+    @Override
+    public HTML<T> del(String cdata) {
+      return del().__(cdata).__();
+    }
+  }
+
+  private <T extends __> HEAD<T> head_(T e, boolean inline) {
+    return new HEAD<T>("head", e, opt(true, inline, false)); }
+
+  private <T extends __> BODY<T> body_(T e, boolean inline) {
+    return new BODY<T>("body", e, opt(true, inline, false)); }
+
+  public class SCRIPT<T extends __> extends EImp<T> implements HamletSpec.SCRIPT {
+    public SCRIPT(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public SCRIPT<T> $type(String value) {
+      addAttr("type", value);
+      return this;
+    }
+
+    @Override
+    public SCRIPT<T> $src(String value) {
+      addAttr("src", value);
+      return this;
+    }
+
+    @Override
+    public SCRIPT<T> $charset(String value) {
+      addAttr("charset", value);
+      return this;
+    }
+
+    @Override
+    public SCRIPT<T> $defer(String value) {
+      addAttr("defer", value);
+      return this;
+    }
+
+    @Override
+    public SCRIPT<T> __(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+  }
+
+  public class STYLE<T extends __> extends EImp<T> implements HamletSpec.STYLE {
+    public STYLE(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public STYLE<T> $type(String value) {
+      addAttr("type", value);
+      return this;
+    }
+
+    @Override
+    public STYLE<T> $media(EnumSet<Media> value) {
+      addMediaAttr("media", value);
+      return this;
+    }
+
+    @Override
+    public STYLE<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public STYLE<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public STYLE<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public STYLE<T> __(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+  }
+
+  public class META<T extends __> extends EImp<T> implements HamletSpec.META {
+    public META(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public META<T> $http_equiv(String value) {
+      addAttr("http-equiv", value);
+      return this;
+    }
+
+    @Override
+    public META<T> $name(String value) {
+      addAttr("name", value);
+      return this;
+    }
+
+    @Override
+    public META<T> $content(String value) {
+      addAttr("content", value);
+      return this;
+    }
+
+    @Override
+    public META<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public META<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+  }
+
+  public class BASE<T extends __> extends EImp<T> implements HamletSpec.BASE {
+    public BASE(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public BASE<T> $href(String value) {
+      addAttr("href", value);
+      return this;
+    }
+  }
+
+  public class TITLE<T extends __> extends EImp<T> implements HamletSpec.TITLE {
+    public TITLE(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TITLE<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TITLE<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TITLE<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public TITLE<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+  }
+
+  public class HEAD<T extends __> extends EImp<T> implements HamletSpec.HEAD {
+    public HEAD(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public HEAD<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public HEAD<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public BASE<HEAD<T>> base() {
+      closeAttrs();
+      return base_(this, false);
+    }
+
+    @Override
+    public HEAD<T> base(String href) {
+      return base().$href(href).__();
+    }
+
+    @Override
+    public TITLE<HEAD<T>> title() {
+      closeAttrs();
+      return title_(this, false);
+    }
+
+    @Override
+    public HEAD<T> title(String cdata) {
+      return title().__(cdata).__();
+    }
+
+    @Override
+    public STYLE<HEAD<T>> style() {
+      closeAttrs();
+      return style_(this, false);
+    }
+
+    @Override
+    public HEAD<T> style(Object... lines) {
+      return style().$type("text/css").__(lines).__();
+    }
+
+    @Override
+    public LINK<HEAD<T>> link() {
+      closeAttrs();
+      return link_(this, false);
+    }
+
+    @Override
+    public HEAD<T> link(String href) {
+      return setLinkHref(link(), href).__();
+    }
+
+    @Override
+    public META<HEAD<T>> meta() {
+      closeAttrs();
+      return meta_(this, false);
+    }
+
+    @Override
+    public HEAD<T> meta(String name, String content) {
+      return meta().$name(name).$content(content).__();
+    }
+
+    @Override
+    public HEAD<T> meta_http(String header, String content) {
+      return meta().$http_equiv(header).$content(content).__();
+    }
+
+    @Override
+    public SCRIPT<HEAD<T>> script() {
+      closeAttrs();
+      return script_(this, false);
+    }
+
+    @Override
+    public HEAD<T> script(String src) {
+      return setScriptSrc(script(), src).__();
+    }
+
+    @Override
+    public OBJECT<HEAD<T>> object() {
+      closeAttrs();
+      return object_(this, true);
+    }
+
+    @Override
+    public OBJECT<HEAD<T>> object(String selector) {
+      return setSelector(object(), selector);
+    }
+  }
+
+  private <T extends __> BASE<T> base_(T e, boolean inline) {
+    return new BASE<T>("base", e, opt(false, inline, false)); }
+
+  private <T extends __> TITLE<T> title_(T e, boolean inline) {
+    return new TITLE<T>("title", e, opt(true, inline, false)); }
+
+  public class TD<T extends __> extends EImp<T> implements HamletSpec.TD {
+    public TD(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TD<T> $headers(String value) {
+      addAttr("headers", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $scope(Scope value) {
+      addAttr("scope", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $rowspan(int value) {
+      addAttr("rowspan", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $colspan(int value) {
+      addAttr("colspan", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TD<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<TD<T>> table() {
+      closeAttrs();
+      return table_(this, false);
+    }
+
+    @Override
+    public TABLE<TD<T>> table(String selector) {
+      return setSelector(table(), selector);
+    }
+
+    @Override
+    public TD<T> address(String cdata) {
+      return address().__(cdata).__();
+    }
+
+    @Override
+    public ADDRESS<TD<T>> address() {
+      closeAttrs();
+      return address_(this, false);
+    }
+
+    @Override
+    public P<TD<T>> p(String selector) {
+      return setSelector(p(), selector);
+    }
+
+    @Override
+    public P<TD<T>> p() {
+      closeAttrs();
+      return p_(this, false);
+    }
+
+    @Override
+    public TD<T> __(Class<? extends SubView> cls) {
+      _v(cls);
+      return this;
+    }
+
+    @Override
+    public HR<TD<T>> hr() {
+      closeAttrs();
+      return hr_(this, false);
+    }
+
+    @Override
+    public TD<T> hr(String selector) {
+      return setSelector(hr(), selector).__();
+    }
+
+    @Override
+    public DL<TD<T>> dl(String selector) {
+      return setSelector(dl(), selector);
+    }
+
+    @Override
+    public DL<TD<T>> dl() {
+      closeAttrs();
+      return dl_(this, false);
+    }
+
+    @Override
+    public DIV<TD<T>> div(String selector) {
+      return setSelector(div(), selector);
+    }
+
+    @Override
+    public DIV<TD<T>> div() {
+      closeAttrs();
+      return div_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<TD<T>> blockquote() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<TD<T>> bq() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public TD<T> h1(String cdata) {
+      return h1().__(cdata).__();
+    }
+
+    @Override
+    public H1<TD<T>> h1() {
+      closeAttrs();
+      return h1_(this, false);
+    }
+
+    @Override
+    public TD<T> h1(String selector, String cdata) {
+      return setSelector(h1(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TD<T> h2(String cdata) {
+      return h2().__(cdata).__();
+    }
+
+    @Override
+    public H2<TD<T>> h2() {
+      closeAttrs();
+      return h2_(this, false);
+    }
+
+    @Override
+    public TD<T> h2(String selector, String cdata) {
+      return setSelector(h2(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H3<TD<T>> h3() {
+      closeAttrs();
+      return h3_(this, false);
+    }
+
+    @Override
+    public TD<T> h3(String cdata) {
+      return h3().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> h3(String selector, String cdata) {
+      return setSelector(h3(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H4<TD<T>> h4() {
+      closeAttrs();
+      return h4_(this, false);
+    }
+
+    @Override
+    public TD<T> h4(String cdata) {
+      return h4().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> h4(String selector, String cdata) {
+      return setSelector(h4(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H5<TD<T>> h5() {
+      closeAttrs();
+      return h5_(this, false);
+    }
+
+    @Override
+    public TD<T> h5(String cdata) {
+      return h5().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> h5(String selector, String cdata) {
+      return setSelector(h5(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H6<TD<T>> h6() {
+      closeAttrs();
+      return h6_(this, false);
+    }
+
+    @Override
+    public TD<T> h6(String cdata) {
+      return h6().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> h6(String selector, String cdata) {
+      return setSelector(h6(), selector).__(cdata).__();
+    }
+
+    @Override
+    public UL<TD<T>> ul() {
+      closeAttrs();
+      return ul_(this, false);
+    }
+
+    @Override
+    public UL<TD<T>> ul(String selector) {
+      return setSelector(ul(), selector);
+    }
+
+    @Override
+    public OL<TD<T>> ol() {
+      closeAttrs();
+      return ol_(this, false);
+    }
+
+    @Override
+    public OL<TD<T>> ol(String selector) {
+      return setSelector(ol(), selector);
+    }
+
+    @Override
+    public PRE<TD<T>> pre() {
+      closeAttrs();
+      return pre_(this, false);
+    }
+
+    @Override
+    public PRE<TD<T>> pre(String selector) {
+      return setSelector(pre(), selector);
+    }
+
+    @Override
+    public FORM<TD<T>> form() {
+      closeAttrs();
+      return form_(this, false);
+    }
+
+    @Override
+    public FORM<TD<T>> form(String selector) {
+      return setSelector(form(), selector);
+    }
+
+    @Override
+    public FIELDSET<TD<T>> fieldset() {
+      closeAttrs();
+      return fieldset_(this, false);
+    }
+
+    @Override
+    public FIELDSET<TD<T>> fieldset(String selector) {
+      return setSelector(fieldset(), selector);
+    }
+
+    @Override
+    public TD<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public TD<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+
+    @Override
+    public B<TD<T>> b() {
+      closeAttrs();
+      return b_(this, true);
+    }
+
+    @Override
+    public TD<T> b(String cdata) {
+      return b().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> b(String selector, String cdata) {
+      return setSelector(b(), selector).__(cdata).__();
+    }
+
+    @Override
+    public I<TD<T>> i() {
+      closeAttrs();
+      return i_(this, true);
+    }
+
+    @Override
+    public TD<T> i(String cdata) {
+      return i().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> i(String selector, String cdata) {
+      return setSelector(i(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SMALL<TD<T>> small() {
+      closeAttrs();
+      return small_(this, true);
+    }
+
+    @Override
+    public TD<T> small(String cdata) {
+      return small().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> small(String selector, String cdata) {
+      return setSelector(small(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TD<T> em(String cdata) {
+      return em().__(cdata).__();
+    }
+
+    @Override
+    public EM<TD<T>> em() {
+      closeAttrs();
+      return em_(this, true);
+    }
+
+    @Override
+    public TD<T> em(String selector, String cdata) {
+      return setSelector(em(), selector).__(cdata).__();
+    }
+
+    @Override
+    public STRONG<TD<T>> strong() {
+      closeAttrs();
+      return strong_(this, true);
+    }
+
+    @Override
+    public TD<T> strong(String cdata) {
+      return strong().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> strong(String selector, String cdata) {
+      return setSelector(strong(), selector).__(cdata).__();
+    }
+
+    @Override
+    public DFN<TD<T>> dfn() {
+      closeAttrs();
+      return dfn_(this, true);
+    }
+
+    @Override
+    public TD<T> dfn(String cdata) {
+      return dfn().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> dfn(String selector, String cdata) {
+      return setSelector(dfn(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CODE<TD<T>> code() {
+      closeAttrs();
+      return code_(this, true);
+    }
+
+    @Override
+    public TD<T> code(String cdata) {
+      return code().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> code(String selector, String cdata) {
+      return setSelector(code(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TD<T> samp(String cdata) {
+      return samp().__(cdata).__();
+    }
+
+    @Override
+    public SAMP<TD<T>> samp() {
+      closeAttrs();
+      return samp_(this, true);
+    }
+
+    @Override
+    public TD<T> samp(String selector, String cdata) {
+      return setSelector(samp(), selector).__(cdata).__();
+    }
+
+    @Override
+    public KBD<TD<T>> kbd() {
+      closeAttrs();
+      return kbd_(this, true);
+    }
+
+    @Override
+    public TD<T> kbd(String cdata) {
+      return kbd().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> kbd(String selector, String cdata) {
+      return setSelector(kbd(), selector).__(cdata).__();
+    }
+
+    @Override
+    public VAR<TD<T>> var() {
+      closeAttrs();
+      return var_(this, true);
+    }
+
+    @Override
+    public TD<T> var(String cdata) {
+      return var().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> var(String selector, String cdata) {
+      return setSelector(var(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CITE<TD<T>> cite() {
+      closeAttrs();
+      return cite_(this, true);
+    }
+
+    @Override
+    public TD<T> cite(String cdata) {
+      return cite().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> cite(String selector, String cdata) {
+      return setSelector(cite(), selector).__(cdata).__();
+    }
+
+    @Override
+    public ABBR<TD<T>> abbr() {
+      closeAttrs();
+      return abbr_(this, true);
+    }
+
+    @Override
+    public TD<T> abbr(String cdata) {
+      return abbr().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> abbr(String selector, String cdata) {
+      return setSelector(abbr(), selector).__(cdata).__();
+    }
+
+    @Override
+    public A<TD<T>> a() {
+      closeAttrs();
+      return a_(this, true);
+    }
+
+    @Override
+    public A<TD<T>> a(String selector) {
+      return setSelector(a(), selector);
+    }
+
+    @Override
+    public TD<T> a(String href, String anchorText) {
+      return a().$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public TD<T> a(String selector, String href, String anchorText) {
+      return setSelector(a(), selector).$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public IMG<TD<T>> img() {
+      closeAttrs();
+      return img_(this, true);
+    }
+
+    @Override
+    public TD<T> img(String src) {
+      return img().$src(src).__();
+    }
+
+    @Override
+    public OBJECT<TD<T>> object() {
+      closeAttrs();
+      return object_(this, true);
+    }
+
+    @Override
+    public OBJECT<TD<T>> object(String selector) {
+      return setSelector(object(), selector);
+    }
+
+    @Override
+    public SUB<TD<T>> sub() {
+      closeAttrs();
+      return sub_(this, true);
+    }
+
+    @Override
+    public TD<T> sub(String cdata) {
+      return sub().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> sub(String selector, String cdata) {
+      return setSelector(sub(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SUP<TD<T>> sup() {
+      closeAttrs();
+      return sup_(this, true);
+    }
+
+    @Override
+    public TD<T> sup(String cdata) {
+      return sup().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> sup(String selector, String cdata) {
+      return setSelector(sup(), selector).__(cdata).__();
+    }
+
+    @Override
+    public MAP<TD<T>> map() {
+      closeAttrs();
+      return map_(this, true);
+    }
+
+    @Override
+    public MAP<TD<T>> map(String selector) {
+      return setSelector(map(), selector);
+    }
+
+    @Override
+    public TD<T> q(String cdata) {
+      return q().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> q(String selector, String cdata) {
+      return setSelector(q(), selector).__(cdata).__();
+    }
+
+    @Override
+    public Q<TD<T>> q() {
+      closeAttrs();
+      return q_(this, true);
+    }
+
+    @Override
+    public BR<TD<T>> br() {
+      closeAttrs();
+      return br_(this, true);
+    }
+
+    @Override
+    public TD<T> br(String selector) {
+      return setSelector(br(), selector).__();
+    }
+
+    @Override
+    public BDO<TD<T>> bdo() {
+      closeAttrs();
+      return bdo_(this, true);
+    }
+
+    @Override
+    public TD<T> bdo(Dir dir, String cdata) {
+      return bdo().$dir(dir).__(cdata).__();
+    }
+
+    @Override
+    public SPAN<TD<T>> span() {
+      closeAttrs();
+      return span_(this, true);
+    }
+
+    @Override
+    public TD<T> span(String cdata) {
+      return span().__(cdata).__();
+    }
+
+    @Override
+    public TD<T> span(String selector, String cdata) {
+      return setSelector(span(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SCRIPT<TD<T>> script() {
+      closeAttrs();
+      return script_(this, true);
+    }
+
+    @Override
+    public TD<T> script(String src) {
+      return setScriptSrc(script(), src).__();
+    }
+
+    @Override
+    public INS<TD<T>> ins() {
+      closeAttrs();
+      return ins_(this, true);
+    }
+
+    @Override
+    public TD<T> ins(String cdata) {
+      return ins().__(cdata).__();
+    }
+
+    @Override
+    public DEL<TD<T>> del() {
+      closeAttrs();
+      return del_(this, true);
+    }
+
+    @Override
+    public TD<T> del(String cdata) {
+      return del().__(cdata).__();
+    }
+
+    @Override
+    public LABEL<TD<T>> label() {
+      closeAttrs();
+      return label_(this, true);
+    }
+
+    @Override
+    public TD<T> label(String forId, String cdata) {
+      return label().$for(forId).__(cdata).__();
+    }
+
+    @Override
+    public INPUT<TD<T>> input(String selector) {
+      return setSelector(input(), selector);
+    }
+
+    @Override
+    public INPUT<TD<T>> input() {
+      closeAttrs();
+      return input_(this, true);
+    }
+
+    @Override
+    public SELECT<TD<T>> select() {
+      closeAttrs();
+      return select_(this, true);
+    }
+
+    @Override
+    public SELECT<TD<T>> select(String selector) {
+      return setSelector(select(), selector);
+    }
+
+    @Override
+    public TEXTAREA<TD<T>> textarea(String selector) {
+      return setSelector(textarea(), selector);
+    }
+
+    @Override
+    public TEXTAREA<TD<T>> textarea() {
+      closeAttrs();
+      return textarea_(this, true);
+    }
+
+    @Override
+    public TD<T> textarea(String selector, String cdata) {
+      return setSelector(textarea(), selector).__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<TD<T>> button() {
+      closeAttrs();
+      return button_(this, true);
+    }
+
+    @Override
+    public BUTTON<TD<T>> button(String selector) {
+      return setSelector(button(), selector);
+    }
+
+    @Override
+    public TD<T> button(String selector, String cdata) {
+      return setSelector(button(), selector).__(cdata).__();
+    }
+  }
+
+  public class TH<T extends __> extends EImp<T> implements HamletSpec.TH {
+    public TH(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TH<T> $headers(String value) {
+      addAttr("headers", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $scope(Scope value) {
+      addAttr("scope", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $rowspan(int value) {
+      addAttr("rowspan", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $colspan(int value) {
+      addAttr("colspan", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TH<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<TH<T>> table() {
+      closeAttrs();
+      return table_(this, false);
+    }
+
+    @Override
+    public TABLE<TH<T>> table(String selector) {
+      return setSelector(table(), selector);
+    }
+
+    @Override
+    public TH<T> address(String cdata) {
+      return address().__(cdata).__();
+    }
+
+    @Override
+    public ADDRESS<TH<T>> address() {
+      closeAttrs();
+      return address_(this, false);
+    }
+
+    @Override
+    public P<TH<T>> p(String selector) {
+      return setSelector(p(), selector);
+    }
+
+    @Override
+    public P<TH<T>> p() {
+      closeAttrs();
+      return p_(this, false);
+    }
+
+    @Override
+    public TH<T> __(Class<? extends SubView> cls) {
+      _v(cls);
+      return this;
+    }
+
+    @Override
+    public HR<TH<T>> hr() {
+      closeAttrs();
+      return hr_(this, false);
+    }
+
+    @Override
+    public TH<T> hr(String selector) {
+      return setSelector(hr(), selector).__();
+    }
+
+    @Override
+    public DL<TH<T>> dl(String selector) {
+      return setSelector(dl(), selector);
+    }
+
+    @Override
+    public DL<TH<T>> dl() {
+      closeAttrs();
+      return dl_(this, false);
+    }
+
+    @Override
+    public DIV<TH<T>> div(String selector) {
+      return setSelector(div(), selector);
+    }
+
+    @Override
+    public DIV<TH<T>> div() {
+      closeAttrs();
+      return div_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<TH<T>> blockquote() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<TH<T>> bq() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public TH<T> h1(String cdata) {
+      return h1().__(cdata).__();
+    }
+
+    @Override
+    public H1<TH<T>> h1() {
+      closeAttrs();
+      return h1_(this, false);
+    }
+
+    @Override
+    public TH<T> h1(String selector, String cdata) {
+      return setSelector(h1(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TH<T> h2(String cdata) {
+      return h2().__(cdata).__();
+    }
+
+    @Override
+    public H2<TH<T>> h2() {
+      closeAttrs();
+      return h2_(this, false);
+    }
+
+    @Override
+    public TH<T> h2(String selector, String cdata) {
+      return setSelector(h2(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H3<TH<T>> h3() {
+      closeAttrs();
+      return h3_(this, false);
+    }
+
+    @Override
+    public TH<T> h3(String cdata) {
+      return h3().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> h3(String selector, String cdata) {
+      return setSelector(h3(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H4<TH<T>> h4() {
+      closeAttrs();
+      return h4_(this, false);
+    }
+
+    @Override
+    public TH<T> h4(String cdata) {
+      return h4().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> h4(String selector, String cdata) {
+      return setSelector(h4(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H5<TH<T>> h5() {
+      closeAttrs();
+      return h5_(this, false);
+    }
+
+    @Override
+    public TH<T> h5(String cdata) {
+      return h5().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> h5(String selector, String cdata) {
+      return setSelector(h5(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H6<TH<T>> h6() {
+      closeAttrs();
+      return h6_(this, false);
+    }
+
+    @Override
+    public TH<T> h6(String cdata) {
+      return h6().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> h6(String selector, String cdata) {
+      return setSelector(h6(), selector).__(cdata).__();
+    }
+
+    @Override
+    public UL<TH<T>> ul() {
+      closeAttrs();
+      return ul_(this, false);
+    }
+
+    @Override
+    public UL<TH<T>> ul(String selector) {
+      return setSelector(ul(), selector);
+    }
+
+    @Override
+    public OL<TH<T>> ol() {
+      closeAttrs();
+      return ol_(this, false);
+    }
+
+    @Override
+    public OL<TH<T>> ol(String selector) {
+      return setSelector(ol(), selector);
+    }
+
+    @Override
+    public PRE<TH<T>> pre() {
+      closeAttrs();
+      return pre_(this, false);
+    }
+
+    @Override
+    public PRE<TH<T>> pre(String selector) {
+      return setSelector(pre(), selector);
+    }
+
+    @Override
+    public FORM<TH<T>> form() {
+      closeAttrs();
+      return form_(this, false);
+    }
+
+    @Override
+    public FORM<TH<T>> form(String selector) {
+      return setSelector(form(), selector);
+    }
+
+    @Override
+    public FIELDSET<TH<T>> fieldset() {
+      closeAttrs();
+      return fieldset_(this, false);
+    }
+
+    @Override
+    public FIELDSET<TH<T>> fieldset(String selector) {
+      return setSelector(fieldset(), selector);
+    }
+
+    @Override
+    public TH<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public TH<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+
+    @Override
+    public B<TH<T>> b() {
+      closeAttrs();
+      return b_(this, true);
+    }
+
+    @Override
+    public TH<T> b(String cdata) {
+      return b().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> b(String selector, String cdata) {
+      return setSelector(b(), selector).__(cdata).__();
+    }
+
+    @Override
+    public I<TH<T>> i() {
+      closeAttrs();
+      return i_(this, true);
+    }
+
+    @Override
+    public TH<T> i(String cdata) {
+      return i().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> i(String selector, String cdata) {
+      return setSelector(i(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SMALL<TH<T>> small() {
+      closeAttrs();
+      return small_(this, true);
+    }
+
+    @Override
+    public TH<T> small(String cdata) {
+      return small().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> small(String selector, String cdata) {
+      return setSelector(small(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TH<T> em(String cdata) {
+      return em().__(cdata).__();
+    }
+
+    @Override
+    public EM<TH<T>> em() {
+      closeAttrs();
+      return em_(this, true);
+    }
+
+    @Override
+    public TH<T> em(String selector, String cdata) {
+      return setSelector(em(), selector).__(cdata).__();
+    }
+
+    @Override
+    public STRONG<TH<T>> strong() {
+      closeAttrs();
+      return strong_(this, true);
+    }
+
+    @Override
+    public TH<T> strong(String cdata) {
+      return strong().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> strong(String selector, String cdata) {
+      return setSelector(strong(), selector).__(cdata).__();
+    }
+
+    @Override
+    public DFN<TH<T>> dfn() {
+      closeAttrs();
+      return dfn_(this, true);
+    }
+
+    @Override
+    public TH<T> dfn(String cdata) {
+      return dfn().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> dfn(String selector, String cdata) {
+      return setSelector(dfn(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CODE<TH<T>> code() {
+      closeAttrs();
+      return code_(this, true);
+    }
+
+    @Override
+    public TH<T> code(String cdata) {
+      return code().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> code(String selector, String cdata) {
+      return setSelector(code(), selector).__(cdata).__();
+    }
+
+    @Override
+    public TH<T> samp(String cdata) {
+      return samp().__(cdata).__();
+    }
+
+    @Override
+    public SAMP<TH<T>> samp() {
+      closeAttrs();
+      return samp_(this, true);
+    }
+
+    @Override
+    public TH<T> samp(String selector, String cdata) {
+      return setSelector(samp(), selector).__(cdata).__();
+    }
+
+    @Override
+    public KBD<TH<T>> kbd() {
+      closeAttrs();
+      return kbd_(this, true);
+    }
+
+    @Override
+    public TH<T> kbd(String cdata) {
+      return kbd().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> kbd(String selector, String cdata) {
+      return setSelector(kbd(), selector).__(cdata).__();
+    }
+
+    @Override
+    public VAR<TH<T>> var() {
+      closeAttrs();
+      return var_(this, true);
+    }
+
+    @Override
+    public TH<T> var(String cdata) {
+      return var().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> var(String selector, String cdata) {
+      return setSelector(var(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CITE<TH<T>> cite() {
+      closeAttrs();
+      return cite_(this, true);
+    }
+
+    @Override
+    public TH<T> cite(String cdata) {
+      return cite().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> cite(String selector, String cdata) {
+      return setSelector(cite(), selector).__(cdata).__();
+    }
+
+    @Override
+    public ABBR<TH<T>> abbr() {
+      closeAttrs();
+      return abbr_(this, true);
+    }
+
+    @Override
+    public TH<T> abbr(String cdata) {
+      return abbr().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> abbr(String selector, String cdata) {
+      return setSelector(abbr(), selector).__(cdata).__();
+    }
+
+    @Override
+    public A<TH<T>> a() {
+      closeAttrs();
+      return a_(this, true);
+    }
+
+    @Override
+    public A<TH<T>> a(String selector) {
+      return setSelector(a(), selector);
+    }
+
+    @Override
+    public TH<T> a(String href, String anchorText) {
+      return a().$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public TH<T> a(String selector, String href, String anchorText) {
+      return setSelector(a(), selector).$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public IMG<TH<T>> img() {
+      closeAttrs();
+      return img_(this, true);
+    }
+
+    @Override
+    public TH<T> img(String src) {
+      return img().$src(src).__();
+    }
+
+    @Override
+    public OBJECT<TH<T>> object() {
+      closeAttrs();
+      return object_(this, true);
+    }
+
+    @Override
+    public OBJECT<TH<T>> object(String selector) {
+      return setSelector(object(), selector);
+    }
+
+    @Override
+    public SUB<TH<T>> sub() {
+      closeAttrs();
+      return sub_(this, true);
+    }
+
+    @Override
+    public TH<T> sub(String cdata) {
+      return sub().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> sub(String selector, String cdata) {
+      return setSelector(sub(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SUP<TH<T>> sup() {
+      closeAttrs();
+      return sup_(this, true);
+    }
+
+    @Override
+    public TH<T> sup(String cdata) {
+      return sup().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> sup(String selector, String cdata) {
+      return setSelector(sup(), selector).__(cdata).__();
+    }
+
+    @Override
+    public MAP<TH<T>> map() {
+      closeAttrs();
+      return map_(this, true);
+    }
+
+    @Override
+    public MAP<TH<T>> map(String selector) {
+      return setSelector(map(), selector);
+    }
+
+    @Override
+    public TH<T> q(String cdata) {
+      return q().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> q(String selector, String cdata) {
+      return setSelector(q(), selector).__(cdata).__();
+    }
+
+    @Override
+    public Q<TH<T>> q() {
+      closeAttrs();
+      return q_(this, true);
+    }
+
+    @Override
+    public BR<TH<T>> br() {
+      closeAttrs();
+      return br_(this, true);
+    }
+
+    @Override
+    public TH<T> br(String selector) {
+      return setSelector(br(), selector).__();
+    }
+
+    @Override
+    public BDO<TH<T>> bdo() {
+      closeAttrs();
+      return bdo_(this, true);
+    }
+
+    @Override
+    public TH<T> bdo(Dir dir, String cdata) {
+      return bdo().$dir(dir).__(cdata).__();
+    }
+
+    @Override
+    public SPAN<TH<T>> span() {
+      closeAttrs();
+      return span_(this, true);
+    }
+
+    @Override
+    public TH<T> span(String cdata) {
+      return span().__(cdata).__();
+    }
+
+    @Override
+    public TH<T> span(String selector, String cdata) {
+      return setSelector(span(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SCRIPT<TH<T>> script() {
+      closeAttrs();
+      return script_(this, true);
+    }
+
+    @Override
+    public TH<T> script(String src) {
+      return setScriptSrc(script(), src).__();
+    }
+
+    @Override
+    public INS<TH<T>> ins() {
+      closeAttrs();
+      return ins_(this, true);
+    }
+
+    @Override
+    public TH<T> ins(String cdata) {
+      return ins().__(cdata).__();
+    }
+
+    @Override
+    public DEL<TH<T>> del() {
+      closeAttrs();
+      return del_(this, true);
+    }
+
+    @Override
+    public TH<T> del(String cdata) {
+      return del().__(cdata).__();
+    }
+
+    @Override
+    public LABEL<TH<T>> label() {
+      closeAttrs();
+      return label_(this, true);
+    }
+
+    @Override
+    public TH<T> label(String forId, String cdata) {
+      return label().$for(forId).__(cdata).__();
+    }
+
+    @Override
+    public INPUT<TH<T>> input(String selector) {
+      return setSelector(input(), selector);
+    }
+
+    @Override
+    public INPUT<TH<T>> input() {
+      closeAttrs();
+      return input_(this, true);
+    }
+
+    @Override
+    public SELECT<TH<T>> select() {
+      closeAttrs();
+      return select_(this, true);
+    }
+
+    @Override
+    public SELECT<TH<T>> select(String selector) {
+      return setSelector(select(), selector);
+    }
+
+    @Override
+    public TEXTAREA<TH<T>> textarea(String selector) {
+      return setSelector(textarea(), selector);
+    }
+
+    @Override
+    public TEXTAREA<TH<T>> textarea() {
+      closeAttrs();
+      return textarea_(this, true);
+    }
+
+    @Override
+    public TH<T> textarea(String selector, String cdata) {
+      return setSelector(textarea(), selector).__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<TH<T>> button() {
+      closeAttrs();
+      return button_(this, true);
+    }
+
+    @Override
+    public BUTTON<TH<T>> button(String selector) {
+      return setSelector(button(), selector);
+    }
+
+    @Override
+    public TH<T> button(String selector, String cdata) {
+      return setSelector(button(), selector).__(cdata).__();
+    }
+  }
+
+  public class TR<T extends __> extends EImp<T> implements HamletSpec.TR {
+    public TR(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TR<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TR<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TH<TR<T>> th() {
+      closeAttrs();
+      return th_(this, false);
+    }
+
+    @Override
+    public TR<T> th(String cdata) {
+      return th().__(cdata).__();
+    }
+
+    @Override
+    public TR<T> th(String selector, String cdata) {
+      return setSelector(th(), selector).__(cdata).__();
+    }
+
+    public TR<T> th(String selector, String title, String cdata) {
+      return setSelector(th(), selector).$title(title).__(cdata).__();
+    }
+
+    @Override
+    public TD<TR<T>> td() {
+      closeAttrs();
+      return td_(this, false);
+    }
+
+    @Override
+    public TR<T> td(String cdata) {
+      return td().__(cdata).__();
+    }
+
+    @Override
+    public TR<T> td(String selector, String cdata) {
+      return setSelector(td(), selector).__(cdata).__();
+    }
+  }
+
+  private <T extends __> TH<T> th_(T e, boolean inline) {
+    return new TH<T>("th", e, opt(true, inline, false)); }
+
+  private <T extends __> TD<T> td_(T e, boolean inline) {
+    return new TD<T>("td", e, opt(true, inline, false)); }
+
+  public class COL<T extends __> extends EImp<T> implements HamletSpec.COL {
+    public COL(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public COL<T> $span(int value) {
+      addAttr("span", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public COL<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+  }
+
+  public class COLGROUP<T extends __> extends EImp<T> implements HamletSpec.COLGROUP {
+    public COLGROUP(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public COLGROUP<T> $span(int value) {
+      addAttr("span", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public COLGROUP<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public COL<COLGROUP<T>> col() {
+      closeAttrs();
+      return col_(this, false);
+    }
+
+    @Override
+    public COLGROUP<T> col(String selector) {
+      return setSelector(col(), selector).__();
+    }
+  }
+
+  public class TBODY<T extends __> extends EImp<T> implements HamletSpec.TBODY {
+    public TBODY(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TBODY<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TBODY<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TR<TBODY<T>> tr() {
+      closeAttrs();
+      return tr_(this, false);
+    }
+
+    @Override
+    public TR<TBODY<T>> tr(String selector) {
+      return setSelector(tr(), selector);
+    }
+  }
+
+  public class TFOOT<T extends __> extends EImp<T> implements HamletSpec.TFOOT {
+    public TFOOT(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TFOOT<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TFOOT<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TR<TFOOT<T>> tr() {
+      closeAttrs();
+      return tr_(this, false);
+    }
+
+    @Override
+    public TR<TFOOT<T>> tr(String selector) {
+      return setSelector(tr(), selector);
+    }
+  }
+
+  public class THEAD<T extends __> extends EImp<T> implements HamletSpec.THEAD {
+    public THEAD(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public THEAD<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public THEAD<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TR<THEAD<T>> tr() {
+      closeAttrs();
+      return tr_(this, false);
+    }
+
+    @Override
+    public TR<THEAD<T>> tr(String selector) {
+      return setSelector(tr(), selector);
+    }
+  }
+
+  public class CAPTION<T extends __> extends EImp<T> implements HamletSpec.CAPTION {
+    public CAPTION(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public CAPTION<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public CAPTION<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+
+    @Override
+    public B<CAPTION<T>> b() {
+      closeAttrs();
+      return b_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> b(String cdata) {
+      return b().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> b(String selector, String cdata) {
+      return setSelector(b(), selector).__(cdata).__();
+    }
+
+    @Override
+    public I<CAPTION<T>> i() {
+      closeAttrs();
+      return i_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> i(String cdata) {
+      return i().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> i(String selector, String cdata) {
+      return setSelector(i(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SMALL<CAPTION<T>> small() {
+      closeAttrs();
+      return small_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> small(String cdata) {
+      return small().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> small(String selector, String cdata) {
+      return setSelector(small(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> em(String cdata) {
+      return em().__(cdata).__();
+    }
+
+    @Override
+    public EM<CAPTION<T>> em() {
+      closeAttrs();
+      return em_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> em(String selector, String cdata) {
+      return setSelector(em(), selector).__(cdata).__();
+    }
+
+    @Override
+    public STRONG<CAPTION<T>> strong() {
+      closeAttrs();
+      return strong_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> strong(String cdata) {
+      return strong().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> strong(String selector, String cdata) {
+      return setSelector(strong(), selector).__(cdata).__();
+    }
+
+    @Override
+    public DFN<CAPTION<T>> dfn() {
+      closeAttrs();
+      return dfn_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> dfn(String cdata) {
+      return dfn().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> dfn(String selector, String cdata) {
+      return setSelector(dfn(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CODE<CAPTION<T>> code() {
+      closeAttrs();
+      return code_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> code(String cdata) {
+      return code().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> code(String selector, String cdata) {
+      return setSelector(code(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> samp(String cdata) {
+      return samp().__(cdata).__();
+    }
+
+    @Override
+    public SAMP<CAPTION<T>> samp() {
+      closeAttrs();
+      return samp_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> samp(String selector, String cdata) {
+      return setSelector(samp(), selector).__(cdata).__();
+    }
+
+    @Override
+    public KBD<CAPTION<T>> kbd() {
+      closeAttrs();
+      return kbd_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> kbd(String cdata) {
+      return kbd().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> kbd(String selector, String cdata) {
+      return setSelector(kbd(), selector).__(cdata).__();
+    }
+
+    @Override
+    public VAR<CAPTION<T>> var() {
+      closeAttrs();
+      return var_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> var(String cdata) {
+      return var().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> var(String selector, String cdata) {
+      return setSelector(var(), selector).__(cdata).__();
+    }
+
+    @Override
+    public CITE<CAPTION<T>> cite() {
+      closeAttrs();
+      return cite_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> cite(String cdata) {
+      return cite().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> cite(String selector, String cdata) {
+      return setSelector(cite(), selector).__(cdata).__();
+    }
+
+    @Override
+    public ABBR<CAPTION<T>> abbr() {
+      closeAttrs();
+      return abbr_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> abbr(String cdata) {
+      return abbr().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> abbr(String selector, String cdata) {
+      return setSelector(abbr(), selector).__(cdata).__();
+    }
+
+    @Override
+    public A<CAPTION<T>> a() {
+      closeAttrs();
+      return a_(this, true);
+    }
+
+    @Override
+    public A<CAPTION<T>> a(String selector) {
+      return setSelector(a(), selector);
+    }
+
+    @Override
+    public CAPTION<T> a(String href, String anchorText) {
+      return a().$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public CAPTION<T> a(String selector, String href, String anchorText) {
+      return setSelector(a(), selector).$href(href).__(anchorText).__();
+    }
+
+    @Override
+    public IMG<CAPTION<T>> img() {
+      closeAttrs();
+      return img_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> img(String src) {
+      return img().$src(src).__();
+    }
+
+    @Override
+    public OBJECT<CAPTION<T>> object() {
+      closeAttrs();
+      return object_(this, true);
+    }
+
+    @Override
+    public OBJECT<CAPTION<T>> object(String selector) {
+      return setSelector(object(), selector);
+    }
+
+    @Override
+    public SUB<CAPTION<T>> sub() {
+      closeAttrs();
+      return sub_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> sub(String cdata) {
+      return sub().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> sub(String selector, String cdata) {
+      return setSelector(sub(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SUP<CAPTION<T>> sup() {
+      closeAttrs();
+      return sup_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> sup(String cdata) {
+      return sup().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> sup(String selector, String cdata) {
+      return setSelector(sup(), selector).__(cdata).__();
+    }
+
+    @Override
+    public MAP<CAPTION<T>> map() {
+      closeAttrs();
+      return map_(this, true);
+    }
+
+    @Override
+    public MAP<CAPTION<T>> map(String selector) {
+      return setSelector(map(), selector);
+    }
+
+    @Override
+    public CAPTION<T> q(String cdata) {
+      return q().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> q(String selector, String cdata) {
+      return setSelector(q(), selector).__(cdata).__();
+    }
+
+    @Override
+    public Q<CAPTION<T>> q() {
+      closeAttrs();
+      return q_(this, true);
+    }
+
+    @Override
+    public BR<CAPTION<T>> br() {
+      closeAttrs();
+      return br_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> br(String selector) {
+      return setSelector(br(), selector).__();
+    }
+
+    @Override
+    public BDO<CAPTION<T>> bdo() {
+      closeAttrs();
+      return bdo_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> bdo(Dir dir, String cdata) {
+      return bdo().$dir(dir).__(cdata).__();
+    }
+
+    @Override
+    public SPAN<CAPTION<T>> span() {
+      closeAttrs();
+      return span_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> span(String cdata) {
+      return span().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<T> span(String selector, String cdata) {
+      return setSelector(span(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SCRIPT<CAPTION<T>> script() {
+      closeAttrs();
+      return script_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> script(String src) {
+      return setScriptSrc(script(), src).__();
+    }
+
+    @Override
+    public INS<CAPTION<T>> ins() {
+      closeAttrs();
+      return ins_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> ins(String cdata) {
+      return ins().__(cdata).__();
+    }
+
+    @Override
+    public DEL<CAPTION<T>> del() {
+      closeAttrs();
+      return del_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> del(String cdata) {
+      return del().__(cdata).__();
+    }
+
+    @Override
+    public LABEL<CAPTION<T>> label() {
+      closeAttrs();
+      return label_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> label(String forId, String cdata) {
+      return label().$for(forId).__(cdata).__();
+    }
+
+    @Override
+    public INPUT<CAPTION<T>> input(String selector) {
+      return setSelector(input(), selector);
+    }
+
+    @Override
+    public INPUT<CAPTION<T>> input() {
+      closeAttrs();
+      return input_(this, true);
+    }
+
+    @Override
+    public SELECT<CAPTION<T>> select() {
+      closeAttrs();
+      return select_(this, true);
+    }
+
+    @Override
+    public SELECT<CAPTION<T>> select(String selector) {
+      return setSelector(select(), selector);
+    }
+
+    @Override
+    public TEXTAREA<CAPTION<T>> textarea(String selector) {
+      return setSelector(textarea(), selector);
+    }
+
+    @Override
+    public TEXTAREA<CAPTION<T>> textarea() {
+      closeAttrs();
+      return textarea_(this, true);
+    }
+
+    @Override
+    public CAPTION<T> textarea(String selector, String cdata) {
+      return setSelector(textarea(), selector).__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<CAPTION<T>> button() {
+      closeAttrs();
+      return button_(this, true);
+    }
+
+    @Override
+    public BUTTON<CAPTION<T>> button(String selector) {
+      return setSelector(button(), selector);
+    }
+
+    @Override
+    public CAPTION<T> button(String selector, String cdata) {
+      return setSelector(button(), selector).__(cdata).__();
+    }
+  }
+
+  public class TABLE<T extends __> extends EImp<T> implements HamletSpec.TABLE {
+    public TABLE(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public TABLE<T> $id(String value) {
+      addAttr("id", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $class(String value) {
+      addAttr("class", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $title(String value) {
+      addAttr("title", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $style(String value) {
+      addAttr("style", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $lang(String value) {
+      addAttr("lang", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $dir(Dir value) {
+      addAttr("dir", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onclick(String value) {
+      addAttr("onclick", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $ondblclick(String value) {
+      addAttr("ondblclick", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onmousedown(String value) {
+      addAttr("onmousedown", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onmouseup(String value) {
+      addAttr("onmouseup", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onmouseover(String value) {
+      addAttr("onmouseover", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onmousemove(String value) {
+      addAttr("onmousemove", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onmouseout(String value) {
+      addAttr("onmouseout", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onkeypress(String value) {
+      addAttr("onkeypress", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onkeydown(String value) {
+      addAttr("onkeydown", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> $onkeyup(String value) {
+      addAttr("onkeyup", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<T> caption(String cdata) {
+      return caption().__(cdata).__();
+    }
+
+    @Override
+    public CAPTION<TABLE<T>> caption() {
+      closeAttrs();
+      return caption_(this, false);
+    }
+
+    @Override
+    public COLGROUP<TABLE<T>> colgroup() {
+      closeAttrs();
+      return colgroup_(this, false);
+    }
+
+    @Override
+    public THEAD<TABLE<T>> thead(String selector) {
+      return setSelector(thead(), selector);
+    }
+
+    @Override
+    public THEAD<TABLE<T>> thead() {
+      closeAttrs();
+      return thead_(this, false);
+    }
+
+    @Override
+    public TFOOT<TABLE<T>> tfoot() {
+      closeAttrs();
+      return tfoot_(this, false);
+    }
+
+    @Override
+    public TFOOT<TABLE<T>> tfoot(String selector) {
+      return setSelector(tfoot(), selector);
+    }
+
+    @Override
+    public TBODY<TABLE<T>> tbody() {
+      closeAttrs();
+      return tbody_(this, false);
+    }
+
+    @Override
+    public TBODY<TABLE<T>> tbody(String selector) {
+      return setSelector(tbody(), selector);
+    }
+
+    @Override
+    public TR<TABLE<T>> tr() {
+      closeAttrs();
+      return tr_(this, false);
+    }
+
+    @Override
+    public TR<TABLE<T>> tr(String selector) {
+      return setSelector(tr(), selector);
+    }
+
+    @Override
+    public COL<TABLE<T>> col() {
+      closeAttrs();
+      return col_(this, false);
+    }
+
+    @Override
+    public TABLE<T> col(String selector) {
+      return setSelector(col(), selector).__();
+    }
+  }
+
+  private <T extends __> CAPTION<T> caption_(T e, boolean inline) {
+    return new CAPTION<T>("caption", e, opt(true, inline, false)); }
+
+  private <T extends __> COLGROUP<T> colgroup_(T e, boolean inline) {
+    return new COLGROUP<T>("colgroup", e, opt(false, inline, false)); }
+
+  private <T extends __> THEAD<T> thead_(T e, boolean inline) {
+    return new THEAD<T>("thead", e, opt(true, inline, false)); }
+
+  private <T extends __> TFOOT<T> tfoot_(T e, boolean inline) {
+    return new TFOOT<T>("tfoot", e, opt(true, inline, false)); }
+
+  private <T extends __> TBODY<T> tbody_(T e, boolean inline) {
+    return new TBODY<T>("tbody", e, opt(true, inline, false)); }
+
+  private <T extends __> COL<T> col_(T e, boolean inline) {
+    return new COL<T>("col", e, opt(false, inline, false)); }
+
+  private <T extends __> TR<T> tr_(T e, boolean inline) {
+    return new TR<T>("tr", e, opt(true, inline, false)); }
+
+  public class BUTTON<T extends __> extends EImp<T> implements HamletSpec.BUTTON {
+    public BUTTON(String name, T parent, EnumSet<EOpt> opts) {
+      super(name, parent, opts);
+    }
+
+    @Override
+    public BUTTON<T> $type(ButtonType value) {
+      addAttr("type", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $name(String value) {
+      addAttr("name", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $value(String value) {
+      addAttr("value", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $disabled() {
+      addAttr("disabled", null);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $tabindex(int value) {
+      addAttr("tabindex", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $accesskey(String value) {
+      addAttr("accesskey", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $onfocus(String value) {
+      addAttr("onfocus", value);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> $onblur(String value) {
+      addAttr("onblur", value);
+      return this;
+    }
+
+    @Override
+    public TABLE<BUTTON<T>> table() {
+      closeAttrs();
+      return table_(this, false);
+    }
+
+    @Override
+    public TABLE<BUTTON<T>> table(String selector) {
+      return setSelector(table(), selector);
+    }
+
+    @Override
+    public BUTTON<T> address(String cdata) {
+      return address().__(cdata).__();
+    }
+
+    @Override
+    public ADDRESS<BUTTON<T>> address() {
+      closeAttrs();
+      return address_(this, false);
+    }
+
+    @Override
+    public P<BUTTON<T>> p(String selector) {
+      return setSelector(p(), selector);
+    }
+
+    @Override
+    public P<BUTTON<T>> p() {
+      closeAttrs();
+      return p_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> __(Class<? extends SubView> cls) {
+      _v(cls);
+      return this;
+    }
+
+    @Override
+    public HR<BUTTON<T>> hr() {
+      closeAttrs();
+      return hr_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> hr(String selector) {
+      return setSelector(hr(), selector).__();
+    }
+
+    @Override
+    public DL<BUTTON<T>> dl(String selector) {
+      return setSelector(dl(), selector);
+    }
+
+    @Override
+    public DL<BUTTON<T>> dl() {
+      closeAttrs();
+      return dl_(this, false);
+    }
+
+    @Override
+    public DIV<BUTTON<T>> div(String selector) {
+      return setSelector(div(), selector);
+    }
+
+    @Override
+    public DIV<BUTTON<T>> div() {
+      closeAttrs();
+      return div_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<BUTTON<T>> blockquote() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public BLOCKQUOTE<BUTTON<T>> bq() {
+      closeAttrs();
+      return blockquote_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h1(String cdata) {
+      return h1().__(cdata).__();
+    }
+
+    @Override
+    public H1<BUTTON<T>> h1() {
+      closeAttrs();
+      return h1_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h1(String selector, String cdata) {
+      return setSelector(h1(), selector).__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> h2(String cdata) {
+      return h2().__(cdata).__();
+    }
+
+    @Override
+    public H2<BUTTON<T>> h2() {
+      closeAttrs();
+      return h2_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h2(String selector, String cdata) {
+      return setSelector(h2(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H3<BUTTON<T>> h3() {
+      closeAttrs();
+      return h3_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h3(String cdata) {
+      return h3().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> h3(String selector, String cdata) {
+      return setSelector(h3(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H4<BUTTON<T>> h4() {
+      closeAttrs();
+      return h4_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h4(String cdata) {
+      return h4().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> h4(String selector, String cdata) {
+      return setSelector(h4(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H5<BUTTON<T>> h5() {
+      closeAttrs();
+      return h5_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h5(String cdata) {
+      return h5().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> h5(String selector, String cdata) {
+      return setSelector(h5(), selector).__(cdata).__();
+    }
+
+    @Override
+    public H6<BUTTON<T>> h6() {
+      closeAttrs();
+      return h6_(this, false);
+    }
+
+    @Override
+    public BUTTON<T> h6(String cdata) {
+      return h6().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> h6(String selector, String cdata) {
+      return setSelector(h6(), selector).__(cdata).__();
+    }
+
+    @Override
+    public UL<BUTTON<T>> ul() {
+      closeAttrs();
+      return ul_(this, false);
+    }
+
+    @Override
+    public UL<BUTTON<T>> ul(String selector) {
+      return setSelector(ul(), selector);
+    }
+
+    @Override
+    public OL<BUTTON<T>> ol() {
+      closeAttrs();
+      return ol_(this, false);
+    }
+
+    @Override
+    public OL<BUTTON<T>> ol(String selector) {
+      return setSelector(ol(), selector);
+    }
+
+    @Override
+    public PRE<BUTTON<T>> pre() {
+      closeAttrs();
+      return pre_(this, false);
+    }
+
+    @Override
+    public PRE<BUTTON<T>> pre(String selector) {
+      return setSelector(pre(), selector);
+    }
+
+    @Override
+    public BUTTON<T> __(Object... lines) {
+      _p(true, lines);
+      return this;
+    }
+
+    @Override
+    public BUTTON<T> _r(Object... lines) {
+      _p(false, lines);
+      return this;
+    }
+
+    @Override
+    public B<BUTTON<T>> b() {
+      closeAttrs();
+      return b_(this, true);
+    }
+
+    @Override
+    public BUTTON<T> b(String cdata) {
+      return b().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> b(String selector, String cdata) {
+      return setSelector(b(), selector).__(cdata).__();
+    }
+
+    @Override
+    public I<BUTTON<T>> i() {
+      closeAttrs();
+      return i_(this, true);
+    }
+
+    @Override
+    public BUTTON<T> i(String cdata) {
+      return i().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> i(String selector, String cdata) {
+      return setSelector(i(), selector).__(cdata).__();
+    }
+
+    @Override
+    public SMALL<BUTTON<T>> small() {
+      closeAttrs();
+      return small_(this, true);
+    }
+
+    @Override
+    public BUTTON<T> small(String cdata) {
+      return small().__(cdata).__();
+    }
+
+    @Override
+    public BUTTON<T> small(Str

<TRUNCATED>

---------------------------------------------------------------------
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-10881. Federation State Store Driver API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10881. Federation State Store Driver API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: b4b7b152f351c4cb0f9049de651d961627a6ff19
Parents: deb0a67
Author: Inigo <in...@apache.org>
Authored: Wed Mar 29 19:35:06 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 28 09:48:38 2017 -0700

----------------------------------------------------------------------
 .../store/StateStoreUnavailableException.java   |  33 ++++
 .../federation/store/StateStoreUtils.java       |  72 +++++++
 .../store/driver/StateStoreDriver.java          | 172 +++++++++++++++++
 .../driver/StateStoreRecordOperations.java      | 164 ++++++++++++++++
 .../store/driver/impl/StateStoreBaseImpl.java   |  69 +++++++
 .../store/driver/impl/package-info.java         |  39 ++++
 .../federation/store/driver/package-info.java   |  37 ++++
 .../federation/store/protocol/package-info.java |  31 +++
 .../federation/store/records/BaseRecord.java    | 189 +++++++++++++++++++
 .../federation/store/records/QueryResult.java   |  56 ++++++
 .../federation/store/records/package-info.java  |  36 ++++
 11 files changed, 898 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
new file mode 100644
index 0000000..4e6f8c8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUnavailableException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.store;
+
+import java.io.IOException;
+
+/**
+ * Thrown when the state store is not reachable or available. Cached APIs and
+ * queries may succeed. Client should retry again later.
+ */
+public class StateStoreUnavailableException extends IOException {
+
+  private static final long serialVersionUID = 1L;
+
+  public StateStoreUnavailableException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
new file mode 100644
index 0000000..8c681df
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
@@ -0,0 +1,72 @@
+/**
+ * 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.store;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Set of utility functions used to query, create, update and delete data
+ * records in the state store.
+ */
+public final class StateStoreUtils {
+
+  private static final Log LOG = LogFactory.getLog(StateStoreUtils.class);
+
+  private StateStoreUtils() {
+    // Utility class
+  }
+
+  /**
+   * Get the base class for a record class. If we get an implementation of a
+   * record we will return the real parent record class.
+   *
+   * @param clazz Class of the data record to check.
+   * @return Base class for the record.
+   */
+  @SuppressWarnings("unchecked")
+  public static <T extends BaseRecord>
+      Class<? extends BaseRecord> getRecordClass(final Class<T> clazz) {
+
+    // We ignore the Impl classes and go to the super class
+    Class<? extends BaseRecord> actualClazz = clazz;
+    while (actualClazz.getSimpleName().endsWith("Impl")) {
+      actualClazz = (Class<? extends BaseRecord>) actualClazz.getSuperclass();
+    }
+
+    // Check if we went too far
+    if (actualClazz.equals(BaseRecord.class)) {
+      LOG.error("We went too far (" + actualClazz + ") with " + clazz);
+      actualClazz = clazz;
+    }
+    return actualClazz;
+  }
+
+  /**
+   * Get the base class for a record. If we get an implementation of a record we
+   * will return the real parent record class.
+   *
+   * @param record Record to check its main class.
+   * @return Base class for the record.
+   */
+  public static <T extends BaseRecord>
+      Class<? extends BaseRecord> getRecordClass(final T record) {
+    return getRecordClass(record.getClass());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
new file mode 100644
index 0000000..a1527df
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
@@ -0,0 +1,172 @@
+/**
+ * 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.store.driver;
+
+import java.net.InetAddress;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Driver class for an implementation of a {@link StateStoreService}
+ * provider. Driver implementations will extend this class and implement some of
+ * the default methods.
+ */
+public abstract class StateStoreDriver implements StateStoreRecordOperations {
+
+  private static final Log LOG = LogFactory.getLog(StateStoreDriver.class);
+
+
+  /** State Store configuration. */
+  private Configuration conf;
+
+  /** Identifier for the driver. */
+  private String identifier;
+
+
+  /**
+   * Initialize the state store connection.
+   * @param config Configuration for the driver.
+   * @param id Identifier for the driver.
+   * @param records Records that are supported.
+   * @return If initialized and ready, false if failed to initialize driver.
+   */
+  public boolean init(final Configuration config, final String id,
+      final List<Class<? extends BaseRecord>> records) {
+
+    this.conf = config;
+    this.identifier = id;
+
+    if (this.identifier == null) {
+      LOG.warn("The identifier for the State Store connection is not set");
+    }
+
+    // TODO stub
+    return false;
+  }
+
+  /**
+   * Get the State Store configuration.
+   *
+   * @return Configuration for the State Store.
+   */
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
+   * Gets a unique identifier for the running task/process. Typically the
+   * router address.
+   *
+   * @return Unique identifier for the running task.
+   */
+  public String getIdentifier() {
+    return this.identifier;
+  }
+
+  /**
+   * Prepare the driver to access data storage.
+   *
+   * @return True if the driver was successfully initialized. If false is
+   *         returned, the state store will periodically attempt to
+   *         re-initialize the driver and the router will remain in safe mode
+   *         until the driver is initialized.
+   */
+  public abstract boolean initDriver();
+
+  /**
+   * Initialize storage for a single record class.
+   *
+   * @param name String reference of the record class to initialize, used to
+   *             construct paths and file names for the record. Determined by
+   *             configuration settings for the specific driver.
+   * @param clazz Record type corresponding to the provided name.
+   * @return True if successful, false otherwise.
+   */
+  public abstract <T extends BaseRecord> boolean initRecordStorage(
+      String className, Class<T> clazz);
+
+  /**
+   * Check if the driver is currently running and the data store connection is
+   * valid.
+   *
+   * @return True if the driver is initialized and the data store is ready.
+   */
+  public abstract boolean isDriverReady();
+
+  /**
+   * Check if the driver is ready to be used and throw an exception otherwise.
+   *
+   * @throws StateStoreUnavailableException If the driver is not ready.
+   */
+  public void verifyDriverReady() throws StateStoreUnavailableException {
+    if (!isDriverReady()) {
+      String driverName = getDriverName();
+      String hostname = getHostname();
+      throw new StateStoreUnavailableException("State Store driver " +
+          driverName + " in " + hostname + " is not ready.");
+    }
+  }
+
+  /**
+   * Close the State Store driver connection.
+   */
+  public abstract void close() throws Exception;
+
+  /**
+   * Returns the current time synchronization from the underlying store.
+   * Override for stores that supply a current date. The data store driver is
+   * responsible for maintaining the official synchronization time/date for all
+   * distributed components.
+   *
+   * @return Current time stamp, used for all synchronization dates.
+   */
+  public long getTime() {
+    return Time.now();
+  }
+
+  /**
+   * Get the name of the driver implementation for debugging.
+   *
+   * @return Name of the driver implementation.
+   */
+  private String getDriverName() {
+    return this.getClass().getSimpleName();
+  }
+
+  /**
+   * Get the host name of the machine running the driver for debugging.
+   *
+   * @return Host name of the machine running the driver.
+   */
+  private String getHostname() {
+    String hostname = "Unknown";
+    try {
+      hostname = InetAddress.getLocalHost().getHostName();
+    } catch (Exception e) {
+      LOG.error("Cannot get local address", e);
+    }
+    return hostname;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
new file mode 100644
index 0000000..739eeba
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
@@ -0,0 +1,164 @@
+/**
+ * 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.store.driver;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.apache.hadoop.io.retry.AtMostOnce;
+import org.apache.hadoop.io.retry.Idempotent;
+
+/**
+ * Operations for a driver to manage records in the State Store.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface StateStoreRecordOperations {
+
+  /**
+   * Get all records of the requested record class from the data store. To use
+   * the default implementations in this class, getAll must return new instances
+   * of the records on each call. It is recommended to override the default
+   * implementations for better performance.
+   *
+   * @param clazz Class of record to fetch.
+   * @return List of all records that match the clazz.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @Idempotent
+  <T extends BaseRecord> QueryResult<T> get(Class<T> clazz) throws IOException;
+
+  /**
+   * Get all records of the requested record class from the data store. To use
+   * the default implementations in this class, getAll must return new instances
+   * of the records on each call. It is recommended to override the default
+   * implementations for better performance.
+   *
+   * @param clazz Class of record to fetch.
+   * @param sub Sub path.
+   * @return List of all records that match the clazz and the sub path.
+   * @throws IOException
+   */
+  @Idempotent
+  <T extends BaseRecord> QueryResult<T> get(Class<T> clazz, String sub)
+      throws IOException;
+
+  /**
+   * Get a single record from the store that matches the query.
+   *
+   * @param clazz Class of record to fetch.
+   * @param query Map of field names and objects to filter results.
+   * @return A single record matching the query. Null if there are no matching
+   *         records or more than one matching record in the store.
+   * @throws IOException If multiple records match or if the data store cannot
+   *           be queried.
+   */
+  @Idempotent
+  <T extends BaseRecord> T get(Class<T> clazz, Map<String, String> query)
+      throws IOException;
+
+  /**
+   * Get multiple records from the store that match a query. This method
+   * assumes the underlying driver does not support filtering. If the driver
+   * supports filtering it should overwrite this method.
+   *
+   * @param clazz Class of record to fetch.
+   * @param query Map of field names and objects to filter results.
+   * @return Records of type clazz that match the query or empty list if none
+   *         are found.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @Idempotent
+  <T extends BaseRecord> List<T> getMultiple(
+      Class<T> clazz, Map<String, String> query) throws IOException;
+
+  /**
+   * Creates a single record. Optionally updates an existing record with same
+   * primary key.
+   *
+   * @param record The record to insert or update.
+   * @param allowUpdate True if update of exiting record is allowed.
+   * @param errorIfExists True if an error should be returned when inserting
+   *          an existing record. Only used if allowUpdate = false.
+   * @return True if the operation was successful.
+   *
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean put(
+       T record, boolean allowUpdate, boolean errorIfExists) throws IOException;
+
+  /**
+   * Creates multiple records. Optionally updates existing records that have
+   * the same primary key.
+   *
+   * @param records List of data records to update or create. All records must
+   *                be of class clazz.
+   * @param clazz Record class of records.
+   * @param allowUpdate True if update of exiting record is allowed.
+   * @param errorIfExists True if an error should be returned when inserting
+   *          an existing record. Only used if allowUpdate = false.
+   * @return true if all operations were successful.
+   *
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean putAll(
+      List<T> records, boolean allowUpdate, boolean errorIfExists)
+          throws IOException;
+
+  /**
+   * Remove a single record.
+   *
+   * @param record Record to be removed.
+   * @return true If the record was successfully removed. False if the record
+   *              could not be removed or not stored.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean remove(T record) throws IOException;
+
+  /**
+   * Remove all records of this class from the store.
+   *
+   * @param clazz Class of records to remove.
+   * @return True if successful.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> boolean removeAll(Class<T> clazz) throws IOException;
+
+  /**
+   * Remove multiple records of a specific class that match a query. Requires
+   * the getAll implementation to fetch fresh records on each call.
+   *
+   * @param clazz Class of record to remove.
+   * @param filter matching filter to remove.
+   * @return The number of records removed.
+   * @throws IOException Throws exception if unable to query the data store.
+   */
+  @AtMostOnce
+  <T extends BaseRecord> int remove(Class<T> clazz, Map<String, String> filter)
+      throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
new file mode 100644
index 0000000..b711fa9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
@@ -0,0 +1,69 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Base implementation of a State Store driver. It contains default
+ * implementations for the optional functions. These implementations use an
+ * uncached read/write all algorithm for all changes. In most cases it is
+ * recommended to override the optional functions.
+ * <p>
+ * Drivers may optionally override additional routines for performance
+ * optimization, such as custom get/put/remove queries, depending on the
+ * capabilities of the data store.
+ * <p>
+ */
+public abstract class StateStoreBaseImpl extends StateStoreDriver {
+
+  @Override
+  public <T extends BaseRecord> T get(
+      Class<T> clazz, Map<String, String> query) throws IOException {
+    List<T> records = getMultiple(clazz, query);
+    if (records.size() > 1) {
+      throw new IOException("Found more than one object in collection");
+    } else if (records.size() == 1) {
+      return records.get(0);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean put(
+      T record, boolean allowUpdate, boolean errorIfExists) throws IOException {
+    List<T> singletonList = new ArrayList<T>();
+    singletonList.add(record);
+    return putAll(singletonList, allowUpdate, errorIfExists);
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean remove(T record) throws IOException {
+    Map<String, String> primaryKeys = record.getPrimaryKeys();
+    Class<? extends BaseRecord> clazz = StateStoreUtils.getRecordClass(record);
+    return remove(clazz, primaryKeys) == 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
new file mode 100644
index 0000000..a18433e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/package-info.java
@@ -0,0 +1,39 @@
+/**
+ * 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.
+ */
+
+/**
+ * Implementations of state store data providers/drivers. Each driver is
+ * responsible for maintaining, querying, updating and deleting persistent data
+ * records. Data records are defined as subclasses of
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord}.
+ * Each driver implements the
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
+ * StateStoreDriver} interface.
+ * <p>
+ * Currently supported drivers:
+ * <ul>
+ * <li>{@link StateStoreFileImpl} A file-based data storage backend.
+ * <li>{@link StateStoreZooKeeperImpl} A zookeeper based data storage backend.
+ * </ul>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.driver.impl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java
new file mode 100644
index 0000000..da998b5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/package-info.java
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+
+/**
+ * The state store uses modular data storage classes derived from
+ * StateStoreDriver to handle querying, updating and deleting data records. The
+ * data storage driver is initialized and maintained by the StateStoreService.
+ * The state store supports fetching all records of a type, filtering by column
+ * values or fetching a single record by its primary key.
+ * <p>
+ * Each data storage backend is required to implement the methods contained in
+ * the StateStoreDriver interface. These methods allow the querying, updating,
+ * inserting and deleting of data records into the state store.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.driver;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
new file mode 100644
index 0000000..0249d2c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/package-info.java
@@ -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.
+ */
+
+/**
+ * Contains the abstract definitions of the API request and response objects for
+ * the various state store APIs. The state store supports multiple interface
+ * APIs and multiple data records. Each protocol object requires a serialization
+ * implementation, the default is protobuf.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
new file mode 100644
index 0000000..4192a3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.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.hdfs.server.federation.store.records;
+
+import java.util.Map;
+
+import org.apache.hadoop.util.Time;
+
+/**
+ * Abstract base of a data record in the StateStore. All StateStore records are
+ * derived from this class. Data records are persisted in the data store and
+ * are identified by their primary key. Each data record contains:
+ * <ul>
+ * <li>A primary key consisting of a combination of record data fields.
+ * <li>A modification date.
+ * <li>A creation date.
+ * </ul>
+ */
+public abstract class BaseRecord implements Comparable<BaseRecord> {
+
+  /**
+   * Set the modification time for the record.
+   *
+   * @param time Modification time of the record.
+   */
+  public abstract void setDateModified(long time);
+
+  /**
+   * Get the modification time for the record.
+   *
+   * @return Modification time of the record.
+   */
+  public abstract long getDateModified();
+
+  /**
+   * Set the creation time for the record.
+   *
+   * @param time Creation time of the record.
+   */
+  public abstract void setDateCreated(long time);
+
+  /**
+   * Get the creation time for the record.
+   *
+   * @return Creation time of the record
+   */
+  public abstract long getDateCreated();
+
+  /**
+   * Get the expiration time for the record.
+   *
+   * @return Expiration time for the record.
+   */
+  public abstract long getExpirationMs();
+
+  /**
+   * Map of primary key names->values for the record. The primary key can be a
+   * combination of 1-n different State Store serialized values.
+   *
+   * @return Map of key/value pairs that constitute this object's primary key.
+   */
+  public abstract Map<String, String> getPrimaryKeys();
+
+  /**
+   * Initialize the object.
+   */
+  public void init() {
+    // Call this after the object has been constructed
+    initDefaultTimes();
+  }
+
+  /**
+   * Initialize default times. The driver may update these timestamps on insert
+   * and/or update. This should only be called when initializing an object that
+   * is not backed by a data store.
+   */
+  private void initDefaultTimes() {
+    long now = Time.now();
+    this.setDateCreated(now);
+    this.setDateModified(now);
+  }
+
+  /**
+   * Join the primary keys into one single primary key.
+   *
+   * @return A string that is guaranteed to be unique amongst all records of
+   *         this type.
+   */
+  public String getPrimaryKey() {
+    return generateMashupKey(getPrimaryKeys());
+  }
+
+  /**
+   * Generates a cache key from a map of values.
+   *
+   * @param keys Map of values.
+   * @return String mashup of key values.
+   */
+  protected static String generateMashupKey(final Map<String, String> keys) {
+    StringBuilder builder = new StringBuilder();
+    for (Object value : keys.values()) {
+      if (builder.length() > 0) {
+        builder.append("-");
+      }
+      builder.append(value);
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Override equals check to use primary key(s) for comparison.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof BaseRecord)) {
+      return false;
+    }
+
+    BaseRecord baseObject = (BaseRecord) obj;
+    Map<String, String> keyset1 = this.getPrimaryKeys();
+    Map<String, String> keyset2 = baseObject.getPrimaryKeys();
+    return keyset1.equals(keyset2);
+  }
+
+  /**
+   * Override hash code to use primary key(s) for comparison.
+   */
+  @Override
+  public int hashCode() {
+    Map<String, String> keyset = this.getPrimaryKeys();
+    return keyset.hashCode();
+  }
+
+  @Override
+  public int compareTo(BaseRecord record) {
+    if (record == null) {
+      return -1;
+    }
+    // Descending date order
+    return (int) (record.getDateModified() - this.getDateModified());
+  }
+
+  /**
+   * Called when the modification time and current time is available, checks for
+   * expirations.
+   *
+   * @param currentTime The current timestamp in ms from the data store, to be
+   *          compared against the modification and creation dates of the
+   *          object.
+   * @return boolean True if the record has been updated and should be
+   *         committed to the data store. Override for customized behavior.
+   */
+  public boolean checkExpired(long currentTime) {
+    long expiration = getExpirationMs();
+    if (getDateModified() > 0 && expiration > 0) {
+      return (getDateModified() + expiration) < currentTime;
+    }
+    return false;
+  }
+
+  /**
+   * Validates the record. Called when the record is created, populated from the
+   * state store, and before committing to the state store.
+   * @return If the record is valid.
+   */
+  public boolean validate() {
+    return getDateCreated() > 0 && getDateModified() > 0;
+  }
+
+  @Override
+  public String toString() {
+    return getPrimaryKey();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.java
new file mode 100644
index 0000000..64c2c71
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/QueryResult.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.hdfs.server.federation.store.records;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Encapsulates a state store query result that includes a set of records and a
+ * time stamp for the result.
+ */
+public class QueryResult<T extends BaseRecord> {
+
+  /** Data result. */
+  private final List<T> records;
+  /** Time stamp of the data results. */
+  private final long timestamp;
+
+  public QueryResult(final List<T> recs, final long time) {
+    this.records = recs;
+    this.timestamp = time;
+  }
+
+  /**
+   * Get the result of the query.
+   *
+   * @return List of records.
+   */
+  public List<T> getRecords() {
+    return Collections.unmodifiableList(this.records);
+  }
+
+  /**
+   * The timetamp in driver time of this query.
+   *
+   * @return Timestamp in driver time.
+   */
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b4b7b152/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
new file mode 100644
index 0000000..63b13af
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/package-info.java
@@ -0,0 +1,36 @@
+/**
+ * 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.
+ */
+
+/**
+ * Contains the abstract definitions of the state store data records. The state
+ * store supports multiple multiple data records.
+ * <p>
+ * Data records inherit from a common class
+ * {@link org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord
+ * BaseRecord}. Data records are serialized when written to the data store using
+ * a modular serialization implementation. The default is profobuf
+ * serialization. Data is stored as rows of records of the same type with each
+ * data member in a record representing a column.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+package org.apache.hadoop.hdfs.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;


---------------------------------------------------------------------
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: HADOOP-14680. Azure: IndexOutOfBoundsException in BlockBlobInputStream. Contributed by Thomas Marquardt.

Posted by in...@apache.org.
HADOOP-14680. Azure: IndexOutOfBoundsException in BlockBlobInputStream. Contributed by Thomas Marquardt.


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

Branch: refs/heads/HDFS-10467
Commit: a92bf39e2313d4bfccd641ce0ccefe26f4903a69
Parents: f81a4ef
Author: Jitendra Pandey <ji...@apache.org>
Authored: Tue Jul 25 16:26:48 2017 -0700
Committer: Jitendra Pandey <ji...@apache.org>
Committed: Tue Jul 25 16:26:48 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/azure/BlockBlobInputStream.java   |  2 +-
 .../fs/azure/TestBlockBlobInputStream.java      | 50 +++++++++++++++++++-
 2 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a92bf39e/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
index 2ed0686..5542415 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlockBlobInputStream.java
@@ -358,7 +358,7 @@ final class BlockBlobInputStream extends InputStream implements Seekable {
      * Gets the current capacity of the stream.
      */
     public synchronized int capacity() {
-      return length - offset;
+      return length;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a92bf39e/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
index 2db063b..2453584 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
@@ -43,8 +43,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
 
-import static org.junit.Assert.*;
-import static org.junit.Assume.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeNotNull;
 
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
@@ -194,6 +197,49 @@ public class TestBlockBlobInputStream extends AbstractWasbTestBase {
     createTestFileAndSetLength();
   }
 
+  @Test
+  public void test_0200_BasicReadTestV2() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStreamV1
+            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
+
+        FSDataInputStream inputStreamV2
+            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
+    ) {
+      byte[] bufferV1 = new byte[3 * MEGABYTE];
+      byte[] bufferV2 = new byte[bufferV1.length];
+
+      // v1 forward seek and read a kilobyte into first kilobyte of bufferV1
+      inputStreamV1.seek(5 * MEGABYTE);
+      int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, KILOBYTE);
+      assertEquals(numBytesReadV1, KILOBYTE);
+
+      // v2 forward seek and read a kilobyte into first kilobyte of bufferV2
+      inputStreamV2.seek(5 * MEGABYTE);
+      int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, KILOBYTE);
+      assertEquals(numBytesReadV2, KILOBYTE);
+
+      assertArrayEquals(bufferV1, bufferV2);
+
+      int len = MEGABYTE;
+      int offset = bufferV1.length - len;
+
+      // v1 reverse seek and read a megabyte into last megabyte of bufferV1
+      inputStreamV1.seek(3 * MEGABYTE);
+      numBytesReadV1 = inputStreamV1.read(bufferV1, offset, len);
+      assertEquals(numBytesReadV1, len);
+
+      // v2 reverse seek and read a megabyte into last megabyte of bufferV2
+      inputStreamV2.seek(3 * MEGABYTE);
+      numBytesReadV2 = inputStreamV2.read(bufferV2, offset, len);
+      assertEquals(numBytesReadV2, len);
+
+      assertArrayEquals(bufferV1, bufferV2);
+    }
+  }
+
   /**
    * Validates the implementation of InputStream.markSupported.
    * @throws IOException


---------------------------------------------------------------------
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: Addendum for YARN-5548. Use MockRMMemoryStateStore to reduce test failures (Bibin A Chundatt via Varun Saxena)

Posted by in...@apache.org.
Addendum for YARN-5548. Use MockRMMemoryStateStore to reduce test failures (Bibin A Chundatt via Varun Saxena)


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

Branch: refs/heads/HDFS-10467
Commit: 11ece0bda1f6e5dd9d0f828b7c29acacf6087baa
Parents: f66fd11
Author: Varun Saxena <va...@apache.org>
Authored: Fri Jul 28 00:10:45 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Jul 28 00:10:45 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/MockRMMemoryStateStore.java | 32 ++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11ece0bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.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/MockRMMemoryStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java
new file mode 100644
index 0000000..d88ee1e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java
@@ -0,0 +1,32 @@
+/**
+ * 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;
+
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+
+/**
+ * Test helper for MemoryRMStateStore will make sure the event.
+ */
+public class MockRMMemoryStateStore extends MemoryRMStateStore {
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected EventHandler getRMStateStoreEventHandler() {
+    return rmStateStoreEventHandler;
+  }
+}
\ 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: YARN-6102. RMActiveService context to be updated with new RMContext on failover. Contributed by Rohith Sharma K S.

Posted by in...@apache.org.
YARN-6102. RMActiveService context to be updated with new RMContext on failover. Contributed by Rohith Sharma K S.


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

Branch: refs/heads/HDFS-10467
Commit: e3153284288d6cfa7a28511dfefe1c8a7d6b4eda
Parents: 2054324
Author: Sunil G <su...@apache.org>
Authored: Mon Jul 24 10:59:01 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Mon Jul 24 11:39:03 2017 +0530

----------------------------------------------------------------------
 ...ActiveStandbyElectorBasedElectorService.java |  12 +-
 .../server/resourcemanager/AdminService.java    |  71 +++--
 .../CuratorBasedElectorService.java             |  10 +-
 .../resourcemanager/RMActiveServiceContext.java |  36 +--
 .../server/resourcemanager/RMContextImpl.java   | 312 ++++++++++---------
 .../resourcemanager/RMServiceContext.java       | 162 ++++++++++
 .../server/resourcemanager/ResourceManager.java |  35 ++-
 .../metrics/TimelineServiceV2Publisher.java     |   6 +-
 .../RMTimelineCollectorManager.java             |  10 +-
 .../yarn/server/resourcemanager/MockRM.java     |   2 +-
 .../resourcemanager/TestRMEmbeddedElector.java  |   8 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |  16 +-
 .../TestSystemMetricsPublisherForV2.java        |  13 +-
 13 files changed, 451 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.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/ActiveStandbyElectorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
index b59bc25..a8dcda4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
@@ -57,7 +57,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
       new HAServiceProtocol.StateChangeRequestInfo(
           HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC);
 
-  private RMContext rmContext;
+  private ResourceManager rm;
 
   private byte[] localActiveNodeInfo;
   private ActiveStandbyElector elector;
@@ -66,9 +66,9 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
   @VisibleForTesting
   final Object zkDisconnectLock = new Object();
 
-  ActiveStandbyElectorBasedElectorService(RMContext rmContext) {
+  ActiveStandbyElectorBasedElectorService(ResourceManager rm) {
     super(ActiveStandbyElectorBasedElectorService.class.getName());
-    this.rmContext = rmContext;
+    this.rm = rm;
   }
 
   @Override
@@ -140,7 +140,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     cancelDisconnectTimer();
 
     try {
-      rmContext.getRMAdminService().transitionToActive(req);
+      rm.getRMContext().getRMAdminService().transitionToActive(req);
     } catch (Exception e) {
       throw new ServiceFailedException("RM could not transition to Active", e);
     }
@@ -151,7 +151,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     cancelDisconnectTimer();
 
     try {
-      rmContext.getRMAdminService().transitionToStandby(req);
+      rm.getRMContext().getRMAdminService().transitionToStandby(req);
     } catch (Exception e) {
       LOG.error("RM could not transition to Standby", e);
     }
@@ -205,7 +205,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
   @SuppressWarnings(value = "unchecked")
   @Override
   public void notifyFatalError(String errorMessage) {
-    rmContext.getDispatcher().getEventHandler().handle(
+    rm.getRMContext().getDispatcher().getEventHandler().handle(
         new RMFatalEvent(RMFatalEventType.EMBEDDED_ELECTOR_FAILED,
             errorMessage));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.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/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 7571765..3457ae3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -102,7 +102,6 @@ public class AdminService extends CompositeService implements
 
   private static final Log LOG = LogFactory.getLog(AdminService.class);
 
-  private final RMContext rmContext;
   private final ResourceManager rm;
   private String rmId;
 
@@ -123,16 +122,16 @@ public class AdminService extends CompositeService implements
   @VisibleForTesting
   boolean isCentralizedNodeLabelConfiguration = true;
 
-  public AdminService(ResourceManager rm, RMContext rmContext) {
+  public AdminService(ResourceManager rm) {
     super(AdminService.class.getName());
     this.rm = rm;
-    this.rmContext = rmContext;
   }
 
   @Override
   public void serviceInit(Configuration conf) throws Exception {
     autoFailoverEnabled =
-        rmContext.isHAEnabled() && HAUtil.isAutomaticFailoverEnabled(conf);
+        rm.getRMContext().isHAEnabled()
+            && HAUtil.isAutomaticFailoverEnabled(conf);
 
     masterServiceBindAddress = conf.getSocketAddr(
         YarnConfiguration.RM_BIND_HOST,
@@ -189,7 +188,7 @@ public class AdminService extends CompositeService implements
           RMPolicyProvider.getInstance());
     }
 
-    if (rmContext.isHAEnabled()) {
+    if (rm.getRMContext().isHAEnabled()) {
       RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
           ProtobufRpcEngine.class);
 
@@ -265,7 +264,7 @@ public class AdminService extends CompositeService implements
   }
 
   private synchronized boolean isRMActive() {
-    return HAServiceState.ACTIVE == rmContext.getHAServiceState();
+    return HAServiceState.ACTIVE == rm.getRMContext().getHAServiceState();
   }
 
   private void throwStandbyException() throws StandbyException {
@@ -304,7 +303,7 @@ public class AdminService extends CompositeService implements
       // call all refresh*s for active RM to get the updated configurations.
       refreshAll();
     } catch (Exception e) {
-      rmContext
+      rm.getRMContext()
           .getDispatcher()
           .getEventHandler()
           .handle(
@@ -363,7 +362,7 @@ public class AdminService extends CompositeService implements
   @Override
   public synchronized HAServiceStatus getServiceStatus() throws IOException {
     checkAccess("getServiceState");
-    HAServiceState haState = rmContext.getHAServiceState();
+    HAServiceState haState = rm.getRMContext().getHAServiceState();
     HAServiceStatus ret = new HAServiceStatus(haState);
     if (isRMActive() || haState == HAServiceProtocol.HAServiceState.STANDBY) {
       ret.setReadyToBecomeActive();
@@ -395,11 +394,12 @@ public class AdminService extends CompositeService implements
   }
 
   private void refreshQueues() throws IOException, YarnException {
-    rmContext.getScheduler().reinitialize(getConfig(), this.rmContext);
+    rm.getRMContext().getScheduler().reinitialize(getConfig(),
+        this.rm.getRMContext());
     // refresh the reservation system
-    ReservationSystem rSystem = rmContext.getReservationSystem();
+    ReservationSystem rSystem = rm.getRMContext().getReservationSystem();
     if (rSystem != null) {
-      rSystem.reinitialize(getConfig(), rmContext);
+      rSystem.reinitialize(getConfig(), rm.getRMContext());
     }
   }
 
@@ -418,14 +418,14 @@ public class AdminService extends CompositeService implements
               YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
       switch (request.getDecommissionType()) {
       case NORMAL:
-        rmContext.getNodesListManager().refreshNodes(conf);
+        rm.getRMContext().getNodesListManager().refreshNodes(conf);
         break;
       case GRACEFUL:
-        rmContext.getNodesListManager().refreshNodesGracefully(
+        rm.getRMContext().getNodesListManager().refreshNodesGracefully(
             conf, request.getDecommissionTimeout());
         break;
       case FORCEFUL:
-        rmContext.getNodesListManager().refreshNodesForcefully();
+        rm.getRMContext().getNodesListManager().refreshNodesForcefully();
         break;
       }
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
@@ -440,7 +440,7 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
-    rmContext.getNodesListManager().refreshNodes(conf);
+    rm.getRMContext().getNodesListManager().refreshNodes(conf);
   }
 
   @Override
@@ -559,10 +559,11 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
-    rmContext.getClientRMService().refreshServiceAcls(conf, policyProvider);
-    rmContext.getApplicationMasterService().refreshServiceAcls(
+    rm.getRMContext().getClientRMService().refreshServiceAcls(conf,
+        policyProvider);
+    rm.getRMContext().getApplicationMasterService().refreshServiceAcls(
         conf, policyProvider);
-    rmContext.getResourceTrackerService().refreshServiceAcls(
+    rm.getRMContext().getResourceTrackerService().refreshServiceAcls(
         conf, policyProvider);
   }
 
@@ -601,7 +602,7 @@ public class AdminService extends CompositeService implements
     // if any invalid nodes, throw exception instead of partially updating
     // valid nodes.
     for (NodeId nodeId : nodeIds) {
-      RMNode node = this.rmContext.getRMNodes().get(nodeId);
+      RMNode node = this.rm.getRMContext().getRMNodes().get(nodeId);
       if (node == null) {
         LOG.error("Resource update get failed on all nodes due to change "
             + "resource on an unrecognized node: " + nodeId);
@@ -619,14 +620,14 @@ public class AdminService extends CompositeService implements
     for (Map.Entry<NodeId, ResourceOption> entry : nodeResourceMap.entrySet()) {
       ResourceOption newResourceOption = entry.getValue();
       NodeId nodeId = entry.getKey();
-      RMNode node = this.rmContext.getRMNodes().get(nodeId);
+      RMNode node = this.rm.getRMContext().getRMNodes().get(nodeId);
 
       if (node == null) {
         LOG.warn("Resource update get failed on an unrecognized node: " + nodeId);
         allSuccess = false;
       } else {
         // update resource to RMNode
-        this.rmContext.getDispatcher().getEventHandler()
+        this.rm.getRMContext().getDispatcher().getEventHandler()
           .handle(new RMNodeResourceUpdateEvent(nodeId, newResourceOption));
         LOG.info("Update resource on node(" + node.getNodeID()
             + ") with resource(" + newResourceOption.toString() + ")");
@@ -661,7 +662,8 @@ public class AdminService extends CompositeService implements
       DynamicResourceConfiguration newConf;
 
       InputStream drInputStream =
-          this.rmContext.getConfigurationProvider().getConfigurationInputStream(
+          this.rm.getRMContext().getConfigurationProvider()
+              .getConfigurationInputStream(
               configuration, YarnConfiguration.DR_CONFIGURATION_FILE);
 
       if (drInputStream != null) {
@@ -679,7 +681,7 @@ public class AdminService extends CompositeService implements
         updateNodeResource(updateRequest);
       }
       // refresh dynamic resource in ResourceTrackerService
-      this.rmContext.getResourceTrackerService().
+      this.rm.getRMContext().getResourceTrackerService().
           updateDynamicResourceConfiguration(newConf);
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
               "AdminService");
@@ -692,7 +694,8 @@ public class AdminService extends CompositeService implements
   private synchronized Configuration getConfiguration(Configuration conf,
       String... confFileNames) throws YarnException, IOException {
     for (String confFileName : confFileNames) {
-      InputStream confFileInputStream = this.rmContext.getConfigurationProvider()
+      InputStream confFileInputStream =
+          this.rm.getRMContext().getConfigurationProvider()
           .getConfigurationInputStream(conf, confFileName);
       if (confFileInputStream != null) {
         conf.addResource(confFileInputStream);
@@ -746,7 +749,7 @@ public class AdminService extends CompositeService implements
     AddToClusterNodeLabelsResponse response =
         recordFactory.newRecordInstance(AddToClusterNodeLabelsResponse.class);
     try {
-      rmContext.getNodeLabelManager()
+      rm.getRMContext().getNodeLabelManager()
           .addToCluserNodeLabels(request.getNodeLabels());
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
           "AdminService");
@@ -769,7 +772,8 @@ public class AdminService extends CompositeService implements
     RemoveFromClusterNodeLabelsResponse response =
         recordFactory.newRecordInstance(RemoveFromClusterNodeLabelsResponse.class);
     try {
-      rmContext.getNodeLabelManager().removeFromClusterNodeLabels(request.getNodeLabels());
+      rm.getRMContext().getNodeLabelManager()
+          .removeFromClusterNodeLabels(request.getNodeLabels());
       RMAuditLogger
           .logSuccess(user.getShortUserName(), operation, "AdminService");
       return response;
@@ -805,19 +809,20 @@ public class AdminService extends CompositeService implements
         boolean isKnown = false;
         // both active and inactive nodes are recognized as known nodes
         if (requestedNode.getPort() != 0) {
-          if (rmContext.getRMNodes().containsKey(requestedNode)
-              || rmContext.getInactiveRMNodes().containsKey(requestedNode)) {
+          if (rm.getRMContext().getRMNodes().containsKey(requestedNode) || rm
+              .getRMContext().getInactiveRMNodes().containsKey(requestedNode)) {
             isKnown = true;
           }
         } else {
-          for (NodeId knownNode : rmContext.getRMNodes().keySet()) {
+          for (NodeId knownNode : rm.getRMContext().getRMNodes().keySet()) {
             if (knownNode.getHost().equals(requestedNode.getHost())) {
               isKnown = true;
               break;
             }
           }
           if (!isKnown) {
-            for (NodeId knownNode : rmContext.getInactiveRMNodes().keySet()) {
+            for (NodeId knownNode : rm.getRMContext().getInactiveRMNodes()
+                .keySet()) {
               if (knownNode.getHost().equals(requestedNode.getHost())) {
                 isKnown = true;
                 break;
@@ -841,7 +846,7 @@ public class AdminService extends CompositeService implements
       }
     }
     try {
-      rmContext.getNodeLabelManager().replaceLabelsOnNode(
+      rm.getRMContext().getNodeLabelManager().replaceLabelsOnNode(
           request.getNodeToLabels());
       RMAuditLogger
           .logSuccess(user.getShortUserName(), operation, "AdminService");
@@ -878,7 +883,7 @@ public class AdminService extends CompositeService implements
 
     checkRMStatus(user.getShortUserName(), operation, msg);
 
-    Set<NodeId> decommissioningNodes = rmContext.getNodesListManager()
+    Set<NodeId> decommissioningNodes = rm.getRMContext().getNodesListManager()
         .checkForDecommissioningNodes();
     RMAuditLogger.logSuccess(user.getShortUserName(), operation,
             "AdminService");
@@ -914,6 +919,6 @@ public class AdminService extends CompositeService implements
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
 
-    rmContext.getScheduler().setClusterMaxPriority(conf);
+    rm.getRMContext().getScheduler().setClusterMaxPriority(conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.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/CuratorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
index bcdf48b..d7485f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
@@ -45,14 +45,12 @@ public class CuratorBasedElectorService extends AbstractService
       LogFactory.getLog(CuratorBasedElectorService.class);
   private LeaderLatch leaderLatch;
   private CuratorFramework curator;
-  private RMContext rmContext;
   private String latchPath;
   private String rmId;
   private ResourceManager rm;
 
-  public CuratorBasedElectorService(RMContext rmContext, ResourceManager rm) {
+  public CuratorBasedElectorService(ResourceManager rm) {
     super(CuratorBasedElectorService.class.getName());
-    this.rmContext = rmContext;
     this.rm = rm;
   }
 
@@ -102,7 +100,8 @@ public class CuratorBasedElectorService extends AbstractService
   public void isLeader() {
     LOG.info(rmId + "is elected leader, transitioning to active");
     try {
-      rmContext.getRMAdminService().transitionToActive(
+      rm.getRMContext().getRMAdminService()
+          .transitionToActive(
           new HAServiceProtocol.StateChangeRequestInfo(
               HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC));
     } catch (Exception e) {
@@ -123,7 +122,8 @@ public class CuratorBasedElectorService extends AbstractService
   public void notLeader() {
     LOG.info(rmId + " relinquish leadership");
     try {
-      rmContext.getRMAdminService().transitionToStandby(
+      rm.getRMContext().getRMAdminService()
+          .transitionToStandby(
           new HAServiceProtocol.StateChangeRequestInfo(
               HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC));
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/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 0e305a9..9dc5945 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
@@ -42,20 +42,20 @@ 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.distributed.QueueLimitCalculator;
 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;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
-import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 
 /**
- * The RMActiveServiceContext is the class that maintains all the
- * RMActiveService contexts.This is expected to be used only by ResourceManager
- * and RMContext.
+ * The RMActiveServiceContext is the class that maintains <b>Active</b> service
+ * context. Services that need to run only on the Active RM. This is expected to
+ * be used only by RMContext.
  */
 @Private
 @Unstable
@@ -94,7 +94,6 @@ public class RMActiveServiceContext {
   private NodesListManager nodesListManager;
   private ResourceTrackerService resourceTrackerService;
   private ApplicationMasterService applicationMasterService;
-  private RMTimelineCollectorManager timelineCollectorManager;
 
   private RMNodeLabelsManager nodeLabelManager;
   private RMDelegatedNodeLabelsUpdater rmDelegatedNodeLabelsUpdater;
@@ -107,6 +106,7 @@ public class RMActiveServiceContext {
   private PlacementManager queuePlacementManager = null;
 
   private RMAppLifetimeMonitor rmAppLifetimeMonitor;
+  private QueueLimitCalculator queueLimitCalculator;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -374,19 +374,6 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
-  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
-    return timelineCollectorManager;
-  }
-
-  @Private
-  @Unstable
-  public void setRMTimelineCollectorManager(
-      RMTimelineCollectorManager collectorManager) {
-    this.timelineCollectorManager = collectorManager;
-  }
-
-  @Private
-  @Unstable
   public long getEpoch() {
     return this.epoch;
   }
@@ -483,4 +470,17 @@ public class RMActiveServiceContext {
   public RMAppLifetimeMonitor getRMAppLifetimeMonitor() {
     return this.rmAppLifetimeMonitor;
   }
+
+  @Private
+  @Unstable
+  public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
+    return this.queueLimitCalculator;
+  }
+
+  @Private
+  @Unstable
+  public void setContainerQueueLimitCalculator(
+      QueueLimitCalculator limitCalculator) {
+    this.queueLimitCalculator = limitCalculator;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/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 fb160c4..db2c585 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
@@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -57,37 +56,39 @@ import org.apache.hadoop.yarn.util.Clock;
 
 import com.google.common.annotations.VisibleForTesting;
 
+/**
+ * RMContextImpl class holds two services context.
+ * <ul>
+ * <li>serviceContext : These services called as <b>Always On</b> services.
+ * Services that need to run always irrespective of the HA state of the RM.</li>
+ * <li>activeServiceCotext : Active services context. Services that need to run
+ * only on the Active RM.</li>
+ * </ul>
+ * <p>
+ * <b>Note:</b> If any new service to be added to context, add it to a right
+ * context as per above description.
+ */
 public class RMContextImpl implements RMContext {
 
-  private Dispatcher rmDispatcher;
-
-  private boolean isHAEnabled;
-
-  private HAServiceState haServiceState =
-      HAServiceProtocol.HAServiceState.INITIALIZING;
-
-  private AdminService adminService;
-
-  private ConfigurationProvider configurationProvider;
+  /**
+   * RM service contexts which runs through out RM life span. These are created
+   * once during start of RM.
+   */
+  private RMServiceContext serviceContext;
 
+  /**
+   * RM Active service context. This will be recreated for every transition from
+   * ACTIVE->STANDBY.
+   */
   private RMActiveServiceContext activeServiceContext;
 
-  private Configuration yarnConfiguration;
-
-  private RMApplicationHistoryWriter rmApplicationHistoryWriter;
-  private SystemMetricsPublisher systemMetricsPublisher;
-  private EmbeddedElector elector;
-
-  private QueueLimitCalculator queueLimitCalculator;
-
-  private final Object haServiceStateLock = new Object();
-
-  private ResourceManager resourceManager;
   /**
    * Default constructor. To be used in conjunction with setter methods for
    * individual fields.
    */
   public RMContextImpl() {
+    this.serviceContext = new RMServiceContext();
+    this.activeServiceContext = new RMActiveServiceContext();
   }
 
   @VisibleForTesting
@@ -138,19 +139,154 @@ public class RMContextImpl implements RMContext {
       clientToAMTokenSecretManager, null);
   }
 
+  /**
+   * RM service contexts which runs through out JVM life span. These are created
+   * once during start of RM.
+   * @return serviceContext of RM
+   */
+  @Private
+  @Unstable
+  public RMServiceContext getServiceContext() {
+    return serviceContext;
+  }
+
+  /**
+   * <b>Note:</b> setting service context clears all services embedded with it.
+   * @param context rm service context
+   */
+  @Private
+  @Unstable
+  public void setServiceContext(RMServiceContext context) {
+    this.serviceContext = context;
+  }
+
   @Override
-  public Dispatcher getDispatcher() {
-    return this.rmDispatcher;
+  public ResourceManager getResourceManager() {
+    return serviceContext.getResourceManager();
+  }
+
+  public void setResourceManager(ResourceManager rm) {
+    serviceContext.setResourceManager(rm);
+  }
+
+  @Override
+  public EmbeddedElector getLeaderElectorService() {
+    return serviceContext.getLeaderElectorService();
   }
 
   @Override
   public void setLeaderElectorService(EmbeddedElector elector) {
-    this.elector = elector;
+    serviceContext.setLeaderElectorService(elector);
   }
 
   @Override
-  public EmbeddedElector getLeaderElectorService() {
-    return this.elector;
+  public Dispatcher getDispatcher() {
+    return serviceContext.getDispatcher();
+  }
+
+  void setDispatcher(Dispatcher dispatcher) {
+    serviceContext.setDispatcher(dispatcher);
+  }
+
+  @Override
+  public AdminService getRMAdminService() {
+    return serviceContext.getRMAdminService();
+  }
+
+  void setRMAdminService(AdminService adminService) {
+    serviceContext.setRMAdminService(adminService);
+  }
+
+  @Override
+  public boolean isHAEnabled() {
+    return serviceContext.isHAEnabled();
+  }
+
+  void setHAEnabled(boolean isHAEnabled) {
+    serviceContext.setHAEnabled(isHAEnabled);
+  }
+
+  @Override
+  public HAServiceState getHAServiceState() {
+    return serviceContext.getHAServiceState();
+  }
+
+  void setHAServiceState(HAServiceState serviceState) {
+    serviceContext.setHAServiceState(serviceState);
+  }
+
+  @Override
+  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
+    return serviceContext.getRMApplicationHistoryWriter();
+  }
+
+  @Override
+  public void setRMApplicationHistoryWriter(
+      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+    serviceContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
+  }
+
+  @Override
+  public SystemMetricsPublisher getSystemMetricsPublisher() {
+    return serviceContext.getSystemMetricsPublisher();
+  }
+
+  @Override
+  public void setSystemMetricsPublisher(
+      SystemMetricsPublisher metricsPublisher) {
+    serviceContext.setSystemMetricsPublisher(metricsPublisher);
+  }
+
+  @Override
+  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
+    return serviceContext.getRMTimelineCollectorManager();
+  }
+
+  @Override
+  public void setRMTimelineCollectorManager(
+      RMTimelineCollectorManager timelineCollectorManager) {
+    serviceContext.setRMTimelineCollectorManager(timelineCollectorManager);
+  }
+
+  @Override
+  public ConfigurationProvider getConfigurationProvider() {
+    return serviceContext.getConfigurationProvider();
+  }
+
+  public void setConfigurationProvider(
+      ConfigurationProvider configurationProvider) {
+    serviceContext.setConfigurationProvider(configurationProvider);
+  }
+
+  @Override
+  public Configuration getYarnConfiguration() {
+    return serviceContext.getYarnConfiguration();
+  }
+
+  public void setYarnConfiguration(Configuration yarnConfiguration) {
+    serviceContext.setYarnConfiguration(yarnConfiguration);
+  }
+
+  public String getHAZookeeperConnectionState() {
+    return serviceContext.getHAZookeeperConnectionState();
+  }
+
+  // ==========================================================================
+  /**
+   * RM Active service context. This will be recreated for every transition from
+   * ACTIVE to STANDBY.
+   * @return activeServiceContext of active services
+   */
+  @Private
+  @Unstable
+  public RMActiveServiceContext getActiveServiceContext() {
+    return activeServiceContext;
+  }
+
+  @Private
+  @Unstable
+  void setActiveServiceContext(RMActiveServiceContext activeServiceContext) {
+    this.activeServiceContext = activeServiceContext;
   }
 
   @Override
@@ -228,11 +364,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getClientToAMTokenSecretManager();
   }
 
-  @Override
-  public AdminService getRMAdminService() {
-    return this.adminService;
-  }
-
   @VisibleForTesting
   public void setStateStore(RMStateStore store) {
     activeServiceContext.setStateStore(store);
@@ -253,24 +384,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getResourceTrackerService();
   }
 
-  void setHAEnabled(boolean isHAEnabled) {
-    this.isHAEnabled = isHAEnabled;
-  }
-
-  void setHAServiceState(HAServiceState serviceState) {
-    synchronized (haServiceStateLock) {
-      this.haServiceState = serviceState;
-    }
-  }
-
-  void setDispatcher(Dispatcher dispatcher) {
-    this.rmDispatcher = dispatcher;
-  }
-
-  void setRMAdminService(AdminService adminService) {
-    this.adminService = adminService;
-  }
-
   @Override
   public void setClientRMService(ClientRMService clientRMService) {
     activeServiceContext.setClientRMService(clientRMService);
@@ -348,18 +461,6 @@ public class RMContextImpl implements RMContext {
     activeServiceContext.setResourceTrackerService(resourceTrackerService);
   }
 
-  @Override
-  public boolean isHAEnabled() {
-    return isHAEnabled;
-  }
-
-  @Override
-  public HAServiceState getHAServiceState() {
-    synchronized (haServiceStateLock) {
-      return haServiceState;
-    }
-  }
-
   public void setWorkPreservingRecoveryEnabled(boolean enabled) {
     activeServiceContext.setWorkPreservingRecoveryEnabled(enabled);
   }
@@ -369,50 +470,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.isWorkPreservingRecoveryEnabled();
   }
 
-  @Override
-  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
-    return this.rmApplicationHistoryWriter;
-  }
-
-  @Override
-  public void setRMTimelineCollectorManager(
-      RMTimelineCollectorManager timelineCollectorManager) {
-    activeServiceContext.setRMTimelineCollectorManager(
-        timelineCollectorManager);
-  }
-
-  @Override
-  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
-    return activeServiceContext.getRMTimelineCollectorManager();
-  }
-
-  @Override
-  public void setSystemMetricsPublisher(
-      SystemMetricsPublisher metricsPublisher) {
-    this.systemMetricsPublisher = metricsPublisher;
-  }
-
-  @Override
-  public SystemMetricsPublisher getSystemMetricsPublisher() {
-    return this.systemMetricsPublisher;
-  }
-
-  @Override
-  public void setRMApplicationHistoryWriter(
-      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
-    this.rmApplicationHistoryWriter = rmApplicationHistoryWriter;
-
-  }
-
-  @Override
-  public ConfigurationProvider getConfigurationProvider() {
-    return this.configurationProvider;
-  }
-
-  public void setConfigurationProvider(
-      ConfigurationProvider configurationProvider) {
-    this.configurationProvider = configurationProvider;
-  }
 
   @Override
   public long getEpoch() {
@@ -463,27 +520,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getSystemCredentialsForApps();
   }
 
-  @Private
-  @Unstable
-  public RMActiveServiceContext getActiveServiceContext() {
-    return activeServiceContext;
-  }
-
-  @Private
-  @Unstable
-  void setActiveServiceContext(RMActiveServiceContext activeServiceContext) {
-    this.activeServiceContext = activeServiceContext;
-  }
-
-  @Override
-  public Configuration getYarnConfiguration() {
-    return this.yarnConfiguration;
-  }
-
-  public void setYarnConfiguration(Configuration yarnConfiguration) {
-    this.yarnConfiguration=yarnConfiguration;
-  }
-
   @Override
   public PlacementManager getQueuePlacementManager() {
     return this.activeServiceContext.getQueuePlacementManager();
@@ -496,12 +532,12 @@ public class RMContextImpl implements RMContext {
 
   @Override
   public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
-    return this.queueLimitCalculator;
+    return activeServiceContext.getNodeManagerQueueLimitCalculator();
   }
 
   public void setContainerQueueLimitCalculator(
       QueueLimitCalculator limitCalculator) {
-    this.queueLimitCalculator = limitCalculator;
+    activeServiceContext.setContainerQueueLimitCalculator(limitCalculator);
   }
 
   @Override
@@ -515,21 +551,5 @@ public class RMContextImpl implements RMContext {
     return this.activeServiceContext.getRMAppLifetimeMonitor();
   }
 
-  public String getHAZookeeperConnectionState() {
-    if (elector == null) {
-      return "Could not find leader elector. Verify both HA and automatic " +
-          "failover are enabled.";
-    } else {
-      return elector.getZookeeperConnectionState();
-    }
-  }
-
-  @Override
-  public ResourceManager getResourceManager() {
-    return resourceManager;
-  }
-
-  public void setResourceManager(ResourceManager rm) {
-    this.resourceManager = rm;
-  }
+  // Note: Read java doc before adding any services over here.
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.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/RMServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
new file mode 100644
index 0000000..45c6166
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
@@ -0,0 +1,162 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.yarn.conf.ConfigurationProvider;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
+
+/**
+ * RMServiceContext class maintains "Always On" services. Services that need to
+ * run always irrespective of the HA state of the RM. This is created during
+ * initialization of RMContextImpl.
+ * <p>
+ * <b>Note:</b> If any services to be added in this class, make sure service
+ * will be running always irrespective of the HA state of the RM
+ */
+@Private
+@Unstable
+public class RMServiceContext {
+
+  private Dispatcher rmDispatcher;
+  private boolean isHAEnabled;
+  private HAServiceState haServiceState =
+      HAServiceProtocol.HAServiceState.INITIALIZING;
+  private AdminService adminService;
+  private ConfigurationProvider configurationProvider;
+  private Configuration yarnConfiguration;
+  private RMApplicationHistoryWriter rmApplicationHistoryWriter;
+  private SystemMetricsPublisher systemMetricsPublisher;
+  private EmbeddedElector elector;
+  private final Object haServiceStateLock = new Object();
+  private ResourceManager resourceManager;
+  private RMTimelineCollectorManager timelineCollectorManager;
+
+  public ResourceManager getResourceManager() {
+    return resourceManager;
+  }
+
+  public void setResourceManager(ResourceManager rm) {
+    this.resourceManager = rm;
+  }
+
+  public ConfigurationProvider getConfigurationProvider() {
+    return this.configurationProvider;
+  }
+
+  public void setConfigurationProvider(
+      ConfigurationProvider configurationProvider) {
+    this.configurationProvider = configurationProvider;
+  }
+
+  public Dispatcher getDispatcher() {
+    return this.rmDispatcher;
+  }
+
+  void setDispatcher(Dispatcher dispatcher) {
+    this.rmDispatcher = dispatcher;
+  }
+
+  public EmbeddedElector getLeaderElectorService() {
+    return this.elector;
+  }
+
+  public void setLeaderElectorService(EmbeddedElector embeddedElector) {
+    this.elector = embeddedElector;
+  }
+
+  public AdminService getRMAdminService() {
+    return this.adminService;
+  }
+
+  void setRMAdminService(AdminService service) {
+    this.adminService = service;
+  }
+
+  void setHAEnabled(boolean rmHAEnabled) {
+    this.isHAEnabled = rmHAEnabled;
+  }
+
+  public boolean isHAEnabled() {
+    return isHAEnabled;
+  }
+
+  public HAServiceState getHAServiceState() {
+    synchronized (haServiceStateLock) {
+      return haServiceState;
+    }
+  }
+
+  void setHAServiceState(HAServiceState serviceState) {
+    synchronized (haServiceStateLock) {
+      this.haServiceState = serviceState;
+    }
+  }
+
+  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
+    return this.rmApplicationHistoryWriter;
+  }
+
+  public void setRMApplicationHistoryWriter(
+      RMApplicationHistoryWriter applicationHistoryWriter) {
+    this.rmApplicationHistoryWriter = applicationHistoryWriter;
+  }
+
+  public void setSystemMetricsPublisher(
+      SystemMetricsPublisher metricsPublisher) {
+    this.systemMetricsPublisher = metricsPublisher;
+  }
+
+  public SystemMetricsPublisher getSystemMetricsPublisher() {
+    return this.systemMetricsPublisher;
+  }
+
+  public Configuration getYarnConfiguration() {
+    return this.yarnConfiguration;
+  }
+
+  public void setYarnConfiguration(Configuration yarnConfiguration) {
+    this.yarnConfiguration = yarnConfiguration;
+  }
+
+  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
+    return timelineCollectorManager;
+  }
+
+  public void setRMTimelineCollectorManager(
+      RMTimelineCollectorManager collectorManager) {
+    this.timelineCollectorManager = collectorManager;
+  }
+
+  public String getHAZookeeperConnectionState() {
+    if (elector == null) {
+      return "Could not find leader elector. Verify both HA and automatic "
+          + "failover are enabled.";
+    } else {
+      return elector.getZookeeperConnectionState();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/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 f727f55..b63b60d 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
@@ -115,7 +115,6 @@ import org.eclipse.jetty.webapp.WebAppContext;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
@@ -345,9 +344,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
       this.curator = createAndStartCurator(conf);
-      elector = new CuratorBasedElectorService(rmContext, this);
+      elector = new CuratorBasedElectorService(this);
     } else {
-      elector = new ActiveStandbyElectorBasedElectorService(rmContext);
+      elector = new ActiveStandbyElectorBasedElectorService(this);
     }
     return elector;
   }
@@ -497,7 +496,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   private RMTimelineCollectorManager createRMTimelineCollectorManager() {
-    return new RMTimelineCollectorManager(rmContext);
+    return new RMTimelineCollectorManager(this);
   }
 
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
@@ -508,7 +507,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
         // we're dealing with the v.2.x publisher
         LOG.info("system metrics publisher with the timeline service V2 is " +
             "configured");
-        publisher = new TimelineServiceV2Publisher(rmContext);
+        publisher = new TimelineServiceV2Publisher(
+            rmContext.getRMTimelineCollectorManager());
       } else {
         // we're dealing with the v.1.x publisher
         LOG.info("system metrics publisher with the timeline service V1 is " +
@@ -560,7 +560,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     private ApplicationMasterLauncher applicationMasterLauncher;
     private ContainerAllocationExpirer containerAllocationExpirer;
     private ResourceManager rm;
-    private RMActiveServiceContext activeServiceContext;
     private boolean fromActive = false;
     private StandByTransitionRunnable standByTransitionRunnable;
 
@@ -573,9 +572,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     protected void serviceInit(Configuration configuration) throws Exception {
       standByTransitionRunnable = new StandByTransitionRunnable();
 
-      activeServiceContext = new RMActiveServiceContext();
-      rmContext.setActiveServiceContext(activeServiceContext);
-
       rmSecretManagerService = createRMSecretManagerService();
       addService(rmSecretManagerService);
 
@@ -1149,7 +1145,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
     ClusterMetrics.destroy();
     QueueMetrics.clearQueueMetrics();
     if (initialize) {
-      resetDispatcher();
+      resetRMContext();
       createAndInitActiveServices(true);
     }
   }
@@ -1294,7 +1290,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   protected AdminService createAdminService() {
-    return new AdminService(this, rmContext);
+    return new AdminService(this);
   }
 
   protected RMSecretManagerService createRMSecretManagerService() {
@@ -1417,17 +1413,24 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return dispatcher;
   }
 
-  private void resetDispatcher() {
+  private void resetRMContext() {
+    RMContextImpl rmContextImpl = new RMContextImpl();
+    // transfer service context to new RM service Context
+    rmContextImpl.setServiceContext(rmContext.getServiceContext());
+
+    // reset dispatcher
     Dispatcher dispatcher = setupDispatcher();
-    ((Service)dispatcher).init(this.conf);
-    ((Service)dispatcher).start();
-    removeService((Service)rmDispatcher);
+    ((Service) dispatcher).init(this.conf);
+    ((Service) dispatcher).start();
+    removeService((Service) rmDispatcher);
     // Need to stop previous rmDispatcher before assigning new dispatcher
     // otherwise causes "AsyncDispatcher event handler" thread leak
     ((Service) rmDispatcher).stop();
     rmDispatcher = dispatcher;
     addIfService(rmDispatcher);
-    rmContext.setDispatcher(rmDispatcher);
+    rmContextImpl.setDispatcher(dispatcher);
+
+    rmContext = rmContextImpl;
   }
 
   private void setSchedulerRecoveryStartAndWaitTime(RMState state,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.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/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index a8bf6bd..a3a2ebc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
@@ -75,9 +74,10 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   private RMTimelineCollectorManager rmTimelineCollectorManager;
   private boolean publishContainerEvents;
 
-  public TimelineServiceV2Publisher(RMContext rmContext) {
+  public TimelineServiceV2Publisher(
+      RMTimelineCollectorManager timelineCollectorManager) {
     super("TimelineserviceV2Publisher");
-    rmTimelineCollectorManager = rmContext.getRMTimelineCollectorManager();
+    rmTimelineCollectorManager = timelineCollectorManager;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.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/timelineservice/RMTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
index 64c3749..c980458 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
@@ -41,16 +41,16 @@ public class RMTimelineCollectorManager extends TimelineCollectorManager {
   private static final Log LOG =
       LogFactory.getLog(RMTimelineCollectorManager.class);
 
-  private RMContext rmContext;
+  private ResourceManager rm;
 
-  public RMTimelineCollectorManager(RMContext rmContext) {
+  public RMTimelineCollectorManager(ResourceManager resourceManager) {
     super(RMTimelineCollectorManager.class.getName());
-    this.rmContext = rmContext;
+    this.rm = resourceManager;
   }
 
   @Override
   protected void doPostPut(ApplicationId appId, TimelineCollector collector) {
-    RMApp app = rmContext.getRMApps().get(appId);
+    RMApp app = rm.getRMContext().getRMApps().get(appId);
     if (app == null) {
       throw new YarnRuntimeException(
           "Unable to get the timeline collector context info for a " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/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 23009db..5a215e5 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
@@ -1055,7 +1055,7 @@ public class MockRM extends ResourceManager {
 
   @Override
   protected AdminService createAdminService() {
-    return new AdminService(this, getRMContext()) {
+    return new AdminService(this) {
       @Override
       protected void startServer() {
         // override to not start rpc handler

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.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/TestRMEmbeddedElector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
index 1fe9bbe..140483a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
@@ -122,13 +122,15 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
       throws IOException, InterruptedException {
     AdminService as = mock(AdminService.class);
     RMContext rc = mock(RMContext.class);
+    ResourceManager rm = mock(ResourceManager.class);
     Configuration myConf = new Configuration(conf);
 
     myConf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 50);
+    when(rm.getRMContext()).thenReturn(rc);
     when(rc.getRMAdminService()).thenReturn(as);
 
-    ActiveStandbyElectorBasedElectorService
-        ees = new ActiveStandbyElectorBasedElectorService(rc);
+    ActiveStandbyElectorBasedElectorService ees =
+        new ActiveStandbyElectorBasedElectorService(rm);
     ees.init(myConf);
 
     ees.enterNeutralMode();
@@ -290,7 +292,7 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
 
     @Override
     protected EmbeddedElector createEmbeddedElector() {
-      return new ActiveStandbyElectorBasedElectorService(getRMContext()) {
+      return new ActiveStandbyElectorBasedElectorService(this) {
         @Override
         public void becomeActive() throws
             ServiceFailedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.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/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
index f807217..ec6b1e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
@@ -71,6 +71,7 @@ public class TestRMHA {
   private Log LOG = LogFactory.getLog(TestRMHA.class);
   private Configuration configuration;
   private MockRM rm = null;
+  private MockNM nm = null;
   private RMApp app = null;
   private RMAppAttempt attempt = null;
   private static final String STATE_ERR =
@@ -135,7 +136,7 @@ public class TestRMHA {
 
     try {
       rm.getNewAppId();
-      rm.registerNode("127.0.0.1:1", 2048);
+      nm = rm.registerNode("127.0.0.1:1", 2048);
       app = rm.submitApp(1024);
       attempt = app.getCurrentAppAttempt();
       rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);
@@ -551,6 +552,17 @@ public class TestRMHA {
     verifyClusterMetrics(1, 1, 1, 1, 2048, 1);
     assertEquals(1, rm.getRMContext().getRMNodes().size());
     assertEquals(1, rm.getRMContext().getRMApps().size());
+    Assert.assertNotNull("Node not registered", nm);
+
+    rm.adminService.transitionToStandby(requestInfo);
+    checkMonitorHealth();
+    checkStandbyRMFunctionality();
+    // race condition causes to register/node heartbeat node even after service
+    // is stopping/stopped. New RMContext is being created on every transition
+    // to standby, so metrics should be 0 which indicates new context reference
+    // has taken.
+    nm.registerNode();
+    verifyClusterMetrics(0, 0, 0, 0, 0, 0);
 
     // 3. Create new RM
     rm = new MockRM(conf, memStore) {
@@ -592,7 +604,7 @@ public class TestRMHA {
     rm = new MockRM(configuration) {
       @Override
       protected AdminService createAdminService() {
-        return new AdminService(this, getRMContext()) {
+        return new AdminService(this) {
           int counter = 0;
           @Override
           protected void setConfig(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3153284/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.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/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
index 2d40c91..ec09945 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 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;
@@ -98,10 +99,12 @@ public class TestSystemMetricsPublisherForV2 {
           new Path(testRootDir.getAbsolutePath()), true);
     }
 
+    ResourceManager rm = mock(ResourceManager.class);
     RMContext rmContext = mock(RMContext.class);
     rmAppsMapInContext = new ConcurrentHashMap<ApplicationId, RMApp>();
     when(rmContext.getRMApps()).thenReturn(rmAppsMapInContext);
-    rmTimelineCollectorManager = new RMTimelineCollectorManager(rmContext);
+    when(rm.getRMContext()).thenReturn(rmContext);
+    rmTimelineCollectorManager = new RMTimelineCollectorManager(rm);
     when(rmContext.getRMTimelineCollectorManager()).thenReturn(
         rmTimelineCollectorManager);
 
@@ -113,7 +116,8 @@ public class TestSystemMetricsPublisherForV2 {
 
     dispatcher.init(conf);
     dispatcher.start();
-    metricsPublisher = new TimelineServiceV2Publisher(rmContext) {
+    metricsPublisher =
+        new TimelineServiceV2Publisher(rmTimelineCollectorManager) {
       @Override
       protected Dispatcher getDispatcher() {
         return dispatcher;
@@ -162,7 +166,7 @@ public class TestSystemMetricsPublisherForV2 {
   public void testSystemMetricPublisherInitialization() {
     @SuppressWarnings("resource")
     TimelineServiceV2Publisher publisher =
-        new TimelineServiceV2Publisher(mock(RMContext.class));
+        new TimelineServiceV2Publisher(mock(RMTimelineCollectorManager.class));
     try {
       Configuration conf = getTimelineV2Conf();
       conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED,
@@ -174,7 +178,8 @@ public class TestSystemMetricsPublisherForV2 {
 
       publisher.stop();
 
-      publisher = new TimelineServiceV2Publisher(mock(RMContext.class));
+      publisher = new TimelineServiceV2Publisher(
+          mock(RMTimelineCollectorManager.class));
       conf = getTimelineV2Conf();
       publisher.init(conf);
       assertTrue("Expected to have registered event handlers and set ready to "


---------------------------------------------------------------------
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: YARN-6845. Variable scheduler of FSLeafQueue duplicates the one of its parent FSQueue. (Contributed by Yufei Gu via Daniel Templeton)

Posted by in...@apache.org.
YARN-6845. Variable scheduler of FSLeafQueue duplicates the one of its parent FSQueue.
(Contributed by Yufei Gu via Daniel Templeton)


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

Branch: refs/heads/HDFS-10467
Commit: 10583625c1c803fc243adf6479cb9435af7e72da
Parents: bb30bd3
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Jul 24 13:44:00 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Jul 24 13:44:00 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10583625/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.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/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index 1de0e30..b911a1a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -52,7 +52,6 @@ public class FSLeafQueue extends FSQueue {
   private static final Log LOG = LogFactory.getLog(FSLeafQueue.class.getName());
   private static final List<FSQueue> EMPTY_LIST = Collections.emptyList();
 
-  private FairScheduler scheduler;
   private FSContext context;
 
   // apps that are runnable
@@ -76,7 +75,6 @@ public class FSLeafQueue extends FSQueue {
   public FSLeafQueue(String name, FairScheduler scheduler,
       FSParentQueue parent) {
     super(name, scheduler, parent);
-    this.scheduler = scheduler;
     this.context = scheduler.getContext();
     this.lastTimeAtMinShare = scheduler.getClock().getTime();
     activeUsersManager = new ActiveUsersManager(getMetrics());


---------------------------------------------------------------------
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: HADOOP-14578. Bind IPC connections to kerberos UPN host for proxy users. Contributed by Daryn Sharp.

Posted by in...@apache.org.
HADOOP-14578. Bind IPC connections to kerberos UPN host for proxy users. Contributed by Daryn Sharp.


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

Branch: refs/heads/HDFS-10467
Commit: 27a1a5fde94d4d7ea0ed172635c146d594413781
Parents: a92bf39
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Jul 26 13:12:39 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed Jul 26 13:12:39 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 34 +++++----
 .../java/org/apache/hadoop/ipc/TestIPC.java     | 76 ++++++++++++++++++++
 2 files changed, 96 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27a1a5fd/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 1daf803..c9ac615 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -633,7 +633,8 @@ public class Client implements AutoCloseable {
       return false;
     }
     
-    private synchronized void setupConnection() throws IOException {
+    private synchronized void setupConnection(
+        UserGroupInformation ticket) throws IOException {
       short ioFailures = 0;
       short timeoutFailures = 0;
       while (true) {
@@ -661,24 +662,26 @@ public class Client implements AutoCloseable {
            * client, to ensure Server matching address of the client connection
            * to host name in principal passed.
            */
-          UserGroupInformation ticket = remoteId.getTicket();
+          InetSocketAddress bindAddr = null;
           if (ticket != null && ticket.hasKerberosCredentials()) {
             KerberosInfo krbInfo = 
               remoteId.getProtocol().getAnnotation(KerberosInfo.class);
-            if (krbInfo != null && krbInfo.clientPrincipal() != null) {
-              String host = 
-                SecurityUtil.getHostFromPrincipal(remoteId.getTicket().getUserName());
-              
+            if (krbInfo != null) {
+              String principal = ticket.getUserName();
+              String host = SecurityUtil.getHostFromPrincipal(principal);
               // If host name is a valid local address then bind socket to it
               InetAddress localAddr = NetUtils.getLocalInetAddress(host);
               if (localAddr != null) {
                 this.socket.setReuseAddress(true);
-                this.socket.bind(new InetSocketAddress(localAddr, 0));
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Binding " + principal + " to " + localAddr);
+                }
+                bindAddr = new InetSocketAddress(localAddr, 0);
               }
             }
           }
           
-          NetUtils.connect(this.socket, server, connectionTimeout);
+          NetUtils.connect(this.socket, server, bindAddr, connectionTimeout);
           this.socket.setSoTimeout(soTimeout);
           return;
         } catch (ConnectTimeoutException toe) {
@@ -762,7 +765,14 @@ public class Client implements AutoCloseable {
         AtomicBoolean fallbackToSimpleAuth) {
       if (socket != null || shouldCloseConnection.get()) {
         return;
-      } 
+      }
+      UserGroupInformation ticket = remoteId.getTicket();
+      if (ticket != null) {
+        final UserGroupInformation realUser = ticket.getRealUser();
+        if (realUser != null) {
+          ticket = realUser;
+        }
+      }
       try {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Connecting to "+server);
@@ -774,14 +784,10 @@ public class Client implements AutoCloseable {
         short numRetries = 0;
         Random rand = null;
         while (true) {
-          setupConnection();
+          setupConnection(ticket);
           ipcStreams = new IpcStreams(socket, maxResponseLength);
           writeConnectionHeader(ipcStreams);
           if (authProtocol == AuthProtocol.SASL) {
-            UserGroupInformation ticket = remoteId.getTicket();
-            if (ticket.getRealUser() != null) {
-              ticket = ticket.getRealUser();
-            }
             try {
               authMethod = ticket
                   .doAs(new PrivilegedExceptionAction<AuthMethod>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27a1a5fd/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 3416746..a4577f2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -39,9 +39,12 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -76,6 +79,7 @@ import org.apache.hadoop.ipc.Server.Connection;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
@@ -1484,6 +1488,78 @@ public class TestIPC {
     Assert.fail("didn't get limit exceeded");
   }
 
+  @Test
+  public void testUserBinding() throws Exception {
+    checkUserBinding(false);
+  }
+
+  @Test
+  public void testProxyUserBinding() throws Exception {
+    checkUserBinding(true);
+  }
+
+  private void checkUserBinding(boolean asProxy) throws Exception {
+    Socket s;
+    // don't attempt bind with no service host.
+    s = checkConnect(null, asProxy);
+    Mockito.verify(s, Mockito.never()).bind(Mockito.any(SocketAddress.class));
+
+    // don't attempt bind with service host not belonging to this host.
+    s = checkConnect("1.2.3.4", asProxy);
+    Mockito.verify(s, Mockito.never()).bind(Mockito.any(SocketAddress.class));
+
+    // do attempt bind when service host is this host.
+    InetAddress addr = InetAddress.getLocalHost();
+    s = checkConnect(addr.getHostAddress(), asProxy);
+    Mockito.verify(s).bind(new InetSocketAddress(addr, 0));
+  }
+
+  // dummy protocol that claims to support kerberos.
+  @KerberosInfo(serverPrincipal = "server@REALM")
+  private static class TestBindingProtocol {
+  }
+
+  private Socket checkConnect(String addr, boolean asProxy) throws Exception {
+    // create a fake ugi that claims to have kerberos credentials.
+    StringBuilder principal = new StringBuilder();
+    principal.append("client");
+    if (addr != null) {
+      principal.append("/").append(addr);
+    }
+    principal.append("@REALM");
+    UserGroupInformation ugi =
+        spy(UserGroupInformation.createRemoteUser(principal.toString()));
+    Mockito.doReturn(true).when(ugi).hasKerberosCredentials();
+    if (asProxy) {
+      ugi = UserGroupInformation.createProxyUser("proxy", ugi);
+    }
+
+    // create a mock socket that throws on connect.
+    SocketException expectedConnectEx =
+        new SocketException("Expected connect failure");
+    Socket s = Mockito.mock(Socket.class);
+    SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
+    Mockito.doReturn(s).when(mockFactory).createSocket();
+    doThrow(expectedConnectEx).when(s).connect(
+        Mockito.any(SocketAddress.class), Mockito.anyInt());
+
+    // do a dummy call and expect it to throw an exception on connect.
+    // tests should verify if/how a bind occurred.
+    try (Client client = new Client(LongWritable.class, conf, mockFactory)) {
+      final InetSocketAddress sockAddr = new InetSocketAddress(0);
+      final LongWritable param = new LongWritable(RANDOM.nextLong());
+      final ConnectionId remoteId = new ConnectionId(
+          sockAddr, TestBindingProtocol.class, ugi, 0,
+          RetryPolicies.TRY_ONCE_THEN_FAIL, conf);
+      client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, null);
+      fail("call didn't throw connect exception");
+    } catch (SocketException se) {
+      // ipc layer re-wraps exceptions, so check the cause.
+      Assert.assertSame(expectedConnectEx, se.getCause());
+    }
+    return s;
+  }
+
   private void doIpcVersionTest(
       byte[] requestData,
       byte[] expectedResponse) throws IOException {


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java
index 1d176d4..210cf04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlPage.java
@@ -25,17 +25,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.WebAppException;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 
 /**
  * The parent class of all HTML pages.  Override 
- * {@link #render(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)}
+ * {@link #render(org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.HTML)}
  * to actually render the page.
  */
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public abstract class HtmlPage extends TextView {
 
-  public static class _ implements Hamlet._ {
+  public static class __ implements Hamlet.__ {
   }
 
   public class Page extends Hamlet {
@@ -50,8 +50,8 @@ public abstract class HtmlPage extends TextView {
       setWasInline(context().wasInline());
     }
 
-    public HTML<HtmlPage._> html() {
-      return new HTML<HtmlPage._>("html", null, EnumSet.of(EOpt.ENDTAG));
+    public HTML<HtmlPage.__> html() {
+      return new HTML<HtmlPage.__>("html", null, EnumSet.of(EOpt.ENDTAG));
     }
   }
 
@@ -91,6 +91,6 @@ public abstract class HtmlPage extends TextView {
    * Render the the HTML page.
    * @param html the page to render data to.
    */
-  protected abstract void render(Page.HTML<_> html);
+  protected abstract void render(Page.HTML<__> html);
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/InfoBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/InfoBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/InfoBlock.java
index 9fe67f1..0ad8b3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/InfoBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/InfoBlock.java
@@ -26,11 +26,11 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._ODD;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
 
 import com.google.inject.Inject;
 
@@ -47,7 +47,7 @@ public class InfoBlock extends HtmlBlock {
       div(_INFO_WRAP).
         table(_INFO).
           tr().
-            th().$class(C_TH).$colspan(2)._(info.about())._()._();
+            th().$class(C_TH).$colspan(2).__(info.about()).__().__();
     int i = 0;
     for (ResponseInfo.Item item : info) {
       TR<TABLE<DIV<Hamlet>>> tr = table.
@@ -62,23 +62,23 @@ public class InfoBlock extends HtmlBlock {
         	DIV<TD<TR<TABLE<DIV<Hamlet>>>>> singleLineDiv;
             for ( String line :lines) {
               singleLineDiv = td.div();
-              singleLineDiv._(line);
-              singleLineDiv._();
+              singleLineDiv.__(line);
+              singleLineDiv.__();
             }
           } else {
-            td._(value);
+            td.__(value);
           }
-          td._();
+          td.__();
         } else {
-          tr.td()._r(value)._();
+          tr.td()._r(value).__();
         }
       } else {
         tr.
           td().
-            a(url(item.url), value)._();
+            a(url(item.url), value).__();
       }
-      tr._();
+      tr.__();
     }
-    table._()._();
+    table.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
index 06372e3..46c76d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java
@@ -26,7 +26,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.split;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.HTML;
+import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.HTML;
 
 import com.google.common.collect.Lists;
 
@@ -82,19 +82,19 @@ public class JQueryUI extends HtmlBlock {
     initProgressBars(list);
 
     if (!list.isEmpty()) {
-      html.script().$type("text/javascript")._("$(function() {")
-          ._(list.toArray())._("});")._();
+      html.script().$type("text/javascript").__("$(function() {")
+          .__(list.toArray()).__("});").__();
     }
   }
 
   public static void jsnotice(HTML html) {
     html.
       div("#jsnotice.ui-state-error").
-          _("This page will not function without javascript enabled."
-            + " Please enable javascript on your browser.")._();
+        __("This page will not function without javascript enabled."
+            + " Please enable javascript on your browser.").__();
     html.
       script().$type("text/javascript").
-        _("$('#jsnotice').hide();")._();
+        __("$('#jsnotice').hide();").__();
   }
 
   protected void initAccordions(List<String> list) {
@@ -130,7 +130,7 @@ public class JQueryUI extends HtmlBlock {
         // for inserting stateSaveInit
         int pos = init.indexOf('{') + 1;  
         init = new StringBuffer(init).insert(pos, stateSaveInit).toString(); 
-        list.add(join(id,"DataTable =  $('#", id, "').dataTable(", init,
+        list.add(join(id, "DataTable =  $('#", id, "').dataTable(", init,
                       ").fnSetFilteringDelay(188);"));
         String postInit = $(postInitID(DATATABLES, id));
         if(!postInit.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/LipsumBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/LipsumBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/LipsumBlock.java
index 4781a20..a4b6f63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/LipsumBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/LipsumBlock.java
@@ -27,7 +27,7 @@ public class LipsumBlock extends HtmlBlock {
   public void render(Block html) {
     html.
       p().
-        _("Lorem ipsum dolor sit amet, consectetur adipiscing elit.",
+        __("Lorem ipsum dolor sit amet, consectetur adipiscing elit.",
           "Vivamus eu dui in ipsum tincidunt egestas ac sed nibh.",
           "Praesent quis nisl lorem, nec interdum urna.",
           "Duis sagittis dignissim purus sed sollicitudin.",
@@ -45,6 +45,6 @@ public class LipsumBlock extends HtmlBlock {
           "Proin eu ante nisl, vel porttitor eros.",
           "Aliquam gravida luctus augue, at scelerisque enim consectetur vel.",
           "Donec interdum tempor nisl, quis laoreet enim venenatis eu.",
-          "Quisque elit elit, vulputate eget porta vel, laoreet ac lacus.")._();
+          "Quisque elit elit, vulputate eget porta vel, laoreet ac lacus.").__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/NavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/NavBlock.java
index cdc13eb..a684eee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/NavBlock.java
@@ -30,12 +30,12 @@ public class NavBlock extends HtmlBlock {
         ul().
           li("Item 1").
           li("Item 2").
-          li("...")._().
+          li("...").__().
         h3("Tools").
         ul().
-          li().a("/conf", "Configuration")._().
-          li().a("/stacks", "Thread dump")._().
-          li().a("/logs", "Logs")._().
-          li().a("/jmx?qry=Hadoop:*", "Metrics")._()._()._();
+          li().a("/conf", "Configuration").__().
+          li().a("/stacks", "Thread dump").__().
+          li().a("/logs", "Logs").__().
+          li().a("/jmx?qry=Hadoop:*", "Metrics").__().__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnCssLayout.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnCssLayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnCssLayout.java
index 23aa51b..3e83199 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnCssLayout.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnCssLayout.java
@@ -31,11 +31,11 @@ import org.apache.hadoop.yarn.webapp.SubView;
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public class TwoColumnCssLayout extends HtmlPage {
 
-  @Override protected void render(Page.HTML<_> html) {
+  @Override protected void render(Page.HTML<__> html) {
     preHead(html);
     html.
       title($("title")).
-      link(root_url("static","yarn.css")).
+      link(root_url("static", "yarn.css")).
       style(".main { min-height: 100%; height: auto !important; height: 100%;",
             "  margin: 0 auto -4em; border: 0; }",
             ".footer, .push { height: 4em; clear: both; border: 0 }",
@@ -50,28 +50,28 @@ public class TwoColumnCssLayout extends HtmlPage {
             "  right: 100%; overflow: hidden; }",
             ".leftnav .nav { float: left; width: 11em; position: relative;",
             "  right: 12em; overflow: hidden; }").
-      _(JQueryUI.class);
+        __(JQueryUI.class);
     postHead(html);
     JQueryUI.jsnotice(html);
     html.
       div(".main.ui-widget-content").
-        _(header()).
+        __(header()).
         div(".cmask.leftnav").
           div(".c1right").
             div(".c1wrap").
               div(".content").
-                _(content())._()._().
+        __(content()).__().__().
             div(".nav").
-              _(nav()).
-              div(".push")._()._()._()._()._().
+        __(nav()).
+              div(".push").__().__().__().__().__().
       div(".footer.ui-widget-content").
-        _(footer())._()._();
+        __(footer()).__().__();
   }
 
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
   }
 
-  protected void postHead(Page.HTML<_> html) {
+  protected void postHead(Page.HTML<__> html) {
   }
 
   protected Class<? extends SubView> header() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java
index 4d7752d..fe71395 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/TwoColumnLayout.java
@@ -39,18 +39,18 @@ public class TwoColumnLayout extends HtmlPage {
    * (non-Javadoc)
    * @see org.apache.hadoop.yarn.webapp.view.HtmlPage#render(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void render(Page.HTML<_> html) {
+  @Override protected void render(Page.HTML<__> html) {
     preHead(html);
     html.
       title($(TITLE)).
-      link(root_url("static","yarn.css")).
+      link(root_url("static", "yarn.css")).
       style("#layout { height: 100%; }",
             "#layout thead td { height: 3em; }",
             "#layout #navcell { width: 11em; padding: 0 1em; }",
             "#layout td.content { padding-top: 0 }",
             "#layout tbody { vertical-align: top; }",
             "#layout tfoot td { height: 4em; }").
-      _(JQueryUI.class);
+        __(JQueryUI.class);
     postHead(html);
     JQueryUI.jsnotice(html);
     html.
@@ -58,17 +58,17 @@ public class TwoColumnLayout extends HtmlPage {
         thead().
           tr().
             td().$colspan(2).
-              _(header())._()._()._().
+        __(header()).__().__().__().
         tfoot().
           tr().
             td().$colspan(2).
-              _(footer())._()._()._().
+        __(footer()).__().__().__().
         tbody().
           tr().
             td().$id("navcell").
-              _(nav())._().
+        __(nav()).__().
             td().$class("content").
-              _(content())._()._()._()._()._();
+        __(content()).__().__().__().__().__();
   }
 
   /**
@@ -76,14 +76,14 @@ public class TwoColumnLayout extends HtmlPage {
    * involves setting page variables for Javascript and CSS rendering.
    * @param html the html to use to render. 
    */
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
   }
 
   /**
    * Do what needs to be done after the header is rendered.
    * @param html the html to use to render. 
    */
-  protected void postHead(Page.HTML<_> html) {
+  protected void postHead(Page.HTML<__> html) {
   }
 
   /**
@@ -120,7 +120,7 @@ public class TwoColumnLayout extends HtmlPage {
    * @param tableId the ID of the table to set styles on.
    * @param innerStyles any other styles to add to the table.
    */
-  protected void setTableStyles(Page.HTML<_> html, String tableId,
+  protected void setTableStyles(Page.HTML<__> html, String tableId,
                                 String... innerStyles) {
     List<String> styles = Lists.newArrayList();
     styles.add(join('#', tableId, "_paginate span {font-weight:normal}"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestSubViews.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestSubViews.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestSubViews.java
index 66d9ef2..075bed2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestSubViews.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestSubViews.java
@@ -32,14 +32,14 @@ public class TestSubViews {
 
   static public class MainView extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         body().
           div().
-            _(Sub1.class)._().
+          __(Sub1.class).__().
           div().
             i("inline text").
-            _(Sub2.class)._()._()._();
+          __(Sub2.class).__().__().__();
     }
   }
 
@@ -48,7 +48,7 @@ public class TestSubViews {
     public void render(Block html) {
       html.
         div("#sub1").
-          _("sub1 text")._();
+          __("sub1 text").__();
     }
   }
 
@@ -57,7 +57,7 @@ public class TestSubViews {
     public void render(Block html) {
       html.
         pre().
-          _("sub2 text")._();
+          __("sub2 text").__();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java
index db50dd3..dea146d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/TestWebApp.java
@@ -100,7 +100,7 @@ public class TestWebApp {
 
   static class TablesView extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       set(DATATABLES_ID, "t1 t2 t3 t4");
       set(initID(DATATABLES, "t1"), tableInit().append("}").toString());
       set(initID(DATATABLES, "t2"), join("{bJQueryUI:true, sDom:'t',",
@@ -110,7 +110,7 @@ public class TestWebApp {
       html.
         title("Test DataTables").
         link("/static/yarn.css").
-        _(JQueryUI.class).
+          __(JQueryUI.class).
         style(".wrapper { padding: 1em }",
               ".wrapper h2 { margin: 0.5em 0 }",
               ".dataTables_wrapper { min-height: 1em }").
@@ -118,33 +118,33 @@ public class TestWebApp {
           h2("Default table init").
           table("#t1").
             thead().
-              tr().th("Column1").th("Column2")._()._().
+              tr().th("Column1").th("Column2").__().__().
             tbody().
-              tr().td("c1r1").td("c2r1")._().
-              tr().td("c1r2").td("c2r2")._()._()._().
+              tr().td("c1r1").td("c2r1").__().
+              tr().td("c1r2").td("c2r2").__().__().__().
           h2("Nested tables").
           div(_INFO_WRAP).
             table("#t2").
               thead().
-                tr().th(_TH, "Column1").th(_TH, "Column2")._()._().
+                tr().th(_TH, "Column1").th(_TH, "Column2").__().__().
               tbody().
                 tr().td("r1"). // th wouldn't work as of dt 1.7.5
                   td().$class(C_TABLE).
                     table("#t3").
                       thead().
-                        tr().th("SubColumn1").th("SubColumn2")._()._().
+                        tr().th("SubColumn1").th("SubColumn2").__().__().
                       tbody().
-                        tr().td("subc1r1").td("subc2r1")._().
-                        tr().td("subc1r2").td("subc2r2")._()._()._()._()._().
+                        tr().td("subc1r1").td("subc2r1").__().
+                        tr().td("subc1r2").td("subc2r2").__().__().__().__().__().
                 tr().td("r2"). // ditto
                   td().$class(C_TABLE).
                     table("#t4").
                       thead().
-                        tr().th("SubColumn1").th("SubColumn2")._()._().
+                        tr().th("SubColumn1").th("SubColumn2").__().__().
                       tbody().
-                        tr().td("subc1r1").td("subc2r1")._().
-                        tr().td("subc1r2").td("subc2r2")._().
-                        _()._()._()._()._()._()._()._()._();
+                        tr().td("subc1r1").td("subc2r1").__().
+                        tr().td("subc1r2").td("subc2r2").__().
+          __().__().__().__().__().__().__().__().__();
     }
   }
 
@@ -358,7 +358,7 @@ public class TestWebApp {
       assertEquals("foo", getContent(baseUrl +"test/foo").trim());
       app1 = WebApps.$for("test", this).at(port).start();
       assertEquals(port, app1.getListenerAddress().getPort());
-      app2 = WebApps.$for("test", this).at("0.0.0.0",port, true).start();
+      app2 = WebApps.$for("test", this).at("0.0.0.0", port, true).start();
       assertTrue(app2.getListenerAddress().getPort() > port);
       Configuration conf = new Configuration();
       port =  ServerSocketUtil.waitForPort(47000, 60);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlBlock.java
index 89042c6..e510dd5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlBlock.java
@@ -24,8 +24,6 @@ import java.io.PrintWriter;
 
 import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
-import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
-import org.apache.hadoop.yarn.webapp.view.HtmlPage;
 
 import org.junit.Test;
 import static org.mockito.Mockito.*;
@@ -35,7 +33,7 @@ public class TestHtmlBlock {
     @Override
     public void render(Block html) {
       html.
-        p("#testid")._("test note")._();
+        p("#testid").__("test note").__();
     }
   }
 
@@ -43,16 +41,16 @@ public class TestHtmlBlock {
     @Override
     public void render(Block html) {
       html.
-        p()._("should throw");
+        p().__("should throw");
     }
   }
 
   public static class ShortPage extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         title("short test").
-        _(ShortBlock.class);
+          __(ShortBlock.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlPage.java
index a5a8e1f..beed31f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestHtmlPage.java
@@ -25,7 +25,6 @@ import java.io.PrintWriter;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
-import org.apache.hadoop.yarn.webapp.view.HtmlPage;
 import org.junit.Test;
 
 import static org.mockito.Mockito.*;
@@ -34,19 +33,19 @@ public class TestHtmlPage {
   
   public static class TestView extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         title("test").
-        p("#testid")._("test note")._()._();
+        p("#testid").__("test note").__().__();
     }
   }
 
   public static class ShortView extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         title("short test").
-        p()._("should throw");
+        p().__("should throw");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestInfoBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestInfoBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestInfoBlock.java
index da5efbb..751aa2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestInfoBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestInfoBlock.java
@@ -45,7 +45,7 @@ public class TestInfoBlock {
 
     static {
       resInfo = new ResponseInfo();
-      resInfo._("User_Name", JAVASCRIPT);
+      resInfo.__("User_Name", JAVASCRIPT);
     }
 
     @Override
@@ -68,8 +68,8 @@ public class TestInfoBlock {
 
     static {
       resInfo = new ResponseInfo();
-      resInfo._("Multiple_line_value", "This is one line.");
-      resInfo._("Multiple_line_value", "This is first line.\nThis is second line.");	
+      resInfo.__("Multiple_line_value", "This is one line.");
+      resInfo.__("Multiple_line_value", "This is first line.\nThis is second line.");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestTwoColumnCssPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestTwoColumnCssPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestTwoColumnCssPage.java
index a718636..20df409 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestTwoColumnCssPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestTwoColumnCssPage.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.webapp.Controller;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
-import org.apache.hadoop.yarn.webapp.view.HtmlPage;
-import org.apache.hadoop.yarn.webapp.view.TwoColumnCssLayout;
 import org.junit.Test;
 
 public class TestTwoColumnCssPage {
@@ -52,10 +50,10 @@ public class TestTwoColumnCssPage {
 
   public static class TestView extends HtmlPage {
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         title($("title")).
-        h1($("title"))._();
+        h1($("title")).__();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSErrorsAndWarningsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSErrorsAndWarningsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSErrorsAndWarningsPage.java
index 3798ee5..1601f8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSErrorsAndWarningsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSErrorsAndWarningsPage.java
@@ -34,7 +34,7 @@ public class AHSErrorsAndWarningsPage extends AHSView {
   }
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String title = "Errors and Warnings in the Application History Server";
     setTitle(title);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java
index 8821bc0..d845503 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java
@@ -32,7 +32,7 @@ public class AHSLogsPage extends AHSView {
    * preHead(org.apache.hadoop .yarn.webapp.hamlet.Hamlet.HTML)
    */
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     String logEntity = $(ENTITY_STRING);
     if (logEntity == null || logEntity.isEmpty()) {
       logEntity = $(CONTAINER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
index 65b5ac1..d965eeb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
@@ -37,7 +37,7 @@ public class AHSView extends TwoColumnLayout {
   static final int MAX_FAST_ROWS = 1000; // inline js array
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "apps");
     set(initID(DATATABLES, "apps"), WebPageUtils.appsTableInit(false));
@@ -49,7 +49,7 @@ public class AHSView extends TwoColumnLayout {
     setTitle(sjoin(reqState, "Applications"));
   }
 
-  protected void commonPreHead(Page.HTML<_> html) {
+  protected void commonPreHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutBlock.java
index b2419e9..996568b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutBlock.java
@@ -19,9 +19,7 @@
 package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
 import com.google.inject.Inject;
-import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
-import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.View;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
@@ -38,10 +36,10 @@ public class AboutBlock extends HtmlBlock {
     TimelineAbout tsInfo = TimelineUtils.createTimelineAbout(
         "Timeline Server - Generic History Service UI");
     info("Timeline Server Overview").
-        _("Timeline Server Version:", tsInfo.getTimelineServiceBuildVersion() +
+        __("Timeline Server Version:", tsInfo.getTimelineServiceBuildVersion() +
             " on " + tsInfo.getTimelineServiceVersionBuiltOn()).
-        _("Hadoop Version:", tsInfo.getHadoopBuildVersion() +
+        __("Hadoop Version:", tsInfo.getHadoopBuildVersion() +
             " on " + tsInfo.getHadoopVersionBuiltOn());
-    html._(InfoBlock.class);
+    html.__(InfoBlock.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutPage.java
index b50073a..1df5832 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AboutPage.java
@@ -20,12 +20,9 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
 
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.YarnWebParams;
-
-import static org.apache.hadoop.yarn.util.StringHelper.join;
 
 public class AboutPage extends AHSView {
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(TITLE, "Timeline Server - Generic History Service");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
index a08297d..ec00db6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AppAttemptPage extends AHSView {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String appAttemptId = $(YarnWebParams.APPLICATION_ATTEMPT_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
index c0e1394..32fcc95 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AppPage extends AHSView {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String appId = $(YarnWebParams.APPLICATION_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java
index 1be8a26..8327ee6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class ContainerPage extends AHSView {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String containerId = $(YarnWebParams.CONTAINER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
index 25ee4f0..3ee4dd1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
@@ -23,7 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 public class NavBlock extends HtmlBlock {
@@ -44,34 +44,34 @@ public class NavBlock extends HtmlBlock {
             h3("Application History").
                 ul().
                     li().a(url("about"), "About").
-                    _().
+        __().
                     li().a(url("apps"), "Applications").
                         ul().
                             li().a(url("apps",
                                 YarnApplicationState.FINISHED.toString()),
                                 YarnApplicationState.FINISHED.toString()).
-                            _().
+        __().
                             li().a(url("apps",
                                 YarnApplicationState.FAILED.toString()),
                                 YarnApplicationState.FAILED.toString()).
-                            _().
+        __().
                             li().a(url("apps",
                                 YarnApplicationState.KILLED.toString()),
                                 YarnApplicationState.KILLED.toString()).
-                            _().
-                        _().
-                    _().
-                _();
+        __().
+        __().
+        __().
+        __();
 
     Hamlet.UL<Hamlet.DIV<Hamlet>> tools = nav.h3("Tools").ul();
-    tools.li().a("/conf", "Configuration")._()
-        .li().a("/logs", "Local logs")._()
-        .li().a("/stacks", "Server stacks")._()
-        .li().a("/jmx?qry=Hadoop:*", "Server metrics")._();
+    tools.li().a("/conf", "Configuration").__()
+        .li().a("/logs", "Local logs").__()
+        .li().a("/stacks", "Server stacks").__()
+        .li().a("/jmx?qry=Hadoop:*", "Server metrics").__();
 
     if (addErrorsAndWarningsLink) {
-      tools.li().a(url("errors-and-warnings"), "Errors/Warnings")._();
+      tools.li().a(url("errors-and-warnings"), "Errors/Warnings").__();
     }
-    tools._()._();
+    tools.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 798c372..87c554d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -36,10 +36,9 @@ import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 import com.google.inject.Inject;
@@ -94,7 +93,7 @@ public class AppAttemptBlock extends HtmlBlock {
       String message =
           "Failed to read the application attempt " + appAttemptId + ".";
       LOG.error(message, e);
-      html.p()._(message)._();
+      html.p().__(message).__();
       return;
     }
 
@@ -140,14 +139,14 @@ public class AppAttemptBlock extends HtmlBlock {
     if (exceptionWhenGetContainerReports) {
       html
         .p()
-        ._(
+        .__(
           "Sorry, Failed to get containers for application attempt" + attemptid
-              + ".")._();
+              + ".").__();
       return;
     }
 
     createAttemptHeadRoomTable(html);
-    html._(InfoBlock.class);
+    html.__(InfoBlock.class);
 
     createTablesForAttemptMetrics(html);
 
@@ -155,7 +154,7 @@ public class AppAttemptBlock extends HtmlBlock {
     TBODY<TABLE<Hamlet>> tbody =
         html.table("#containers").thead().tr().th(".id", "Container ID")
           .th(".node", "Node").th(".exitstatus", "Container Exit Status")
-          .th(".logs", "Logs")._()._().tbody();
+          .th(".logs", "Logs").__().__().tbody();
 
     StringBuilder containersTableData = new StringBuilder("[\n");
     for (ContainerReport containerReport : containers) {
@@ -186,9 +185,9 @@ public class AppAttemptBlock extends HtmlBlock {
     }
     containersTableData.append("]");
     html.script().$type("text/javascript")
-      ._("var containersTableData=" + containersTableData)._();
+      .__("var containersTableData=" + containersTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 
   protected void generateOverview(ApplicationAttemptReport appAttemptReport,
@@ -196,18 +195,18 @@ public class AppAttemptBlock extends HtmlBlock {
       String node) {
     String amContainerId = appAttempt.getAmContainerId();
     info("Application Attempt Overview")
-      ._(
+      .__(
         "Application Attempt State:",
         appAttempt.getAppAttemptState() == null ? UNAVAILABLE : appAttempt
           .getAppAttemptState())
-      ._("AM Container:",
+      .__("AM Container:",
           amContainerId == null
               || containers == null
               || !hasAMContainer(appAttemptReport.getAMContainerId(),
                   containers) ? null : root_url("container", amContainerId),
           amContainerId == null ? "N/A" : amContainerId)
-      ._("Node:", node)
-      ._(
+      .__("Node:", node)
+      .__(
         "Tracking URL:",
         appAttempt.getTrackingUrl() == null
             || appAttempt.getTrackingUrl().equals(UNAVAILABLE) ? null
@@ -219,7 +218,7 @@ public class AppAttemptBlock extends HtmlBlock {
                 || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.FAILED
                 || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.KILLED
                 ? "History" : "ApplicationMaster")
-      ._(
+      .__(
         "Diagnostics Info:",
         appAttempt.getDiagnosticsInfo() == null ? "" : appAttempt
           .getDiagnosticsInfo());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 11bd9b4..d4090aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -55,9 +55,9 @@ import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
@@ -116,7 +116,7 @@ public class AppBlock extends HtmlBlock {
     } catch (Exception e) {
       String message = "Failed to read the application " + appID + ".";
       LOG.error(message, e);
-      html.p()._(message)._();
+      html.p().__(message).__();
       return;
     }
 
@@ -136,8 +136,8 @@ public class AppBlock extends HtmlBlock {
       // Application Kill
       html.div()
         .button()
-          .$onclick("confirmAction()").b("Kill Application")._()
-          ._();
+          .$onclick("confirmAction()").b("Kill Application").__()
+          .__();
 
       StringBuilder script = new StringBuilder();
       script.append("function confirmAction() {")
@@ -160,7 +160,7 @@ public class AppBlock extends HtmlBlock {
           .append(" }")
           .append("}");
 
-      html.script().$type("text/javascript")._(script.toString())._();
+      html.script().$type("text/javascript").__(script.toString()).__();
     }
 
     String schedulerPath = WebAppUtils.getResolvedRMWebAppURLWithScheme(conf) +
@@ -185,13 +185,13 @@ public class AppBlock extends HtmlBlock {
       String message =
           "Failed to read the attempts of the application " + appID + ".";
       LOG.error(message, e);
-      html.p()._(message)._();
+      html.p().__(message).__();
       return;
     }
 
     createApplicationMetricsTable(html);
 
-    html._(InfoBlock.class);
+    html.__(InfoBlock.class);
 
     generateApplicationTable(html, callerUGI, attempts);
 
@@ -207,25 +207,25 @@ public class AppBlock extends HtmlBlock {
   private void generateOverviewTable(AppInfo app, String schedulerPath,
       String webUiType, ApplicationReport appReport) {
     ResponseInfo overviewTable = info("Application Overview")
-        ._("User:", schedulerPath, app.getUser())
-        ._("Name:", app.getName())
-        ._("Application Type:", app.getType())
-        ._("Application Tags:",
+        .__("User:", schedulerPath, app.getUser())
+        .__("Name:", app.getName())
+        .__("Application Type:", app.getType())
+        .__("Application Tags:",
             app.getApplicationTags() == null ? "" : app.getApplicationTags())
-        ._("Application Priority:", clarifyAppPriority(app.getPriority()))
-        ._(
+        .__("Application Priority:", clarifyAppPriority(app.getPriority()))
+        .__(
             "YarnApplicationState:",
             app.getAppState() == null ? UNAVAILABLE : clarifyAppState(app
                 .getAppState()))
-        ._("Queue:", schedulerPath, app.getQueue())
-        ._("FinalStatus Reported by AM:",
+        .__("Queue:", schedulerPath, app.getQueue())
+        .__("FinalStatus Reported by AM:",
             clairfyAppFinalStatus(app.getFinalAppStatus()))
-        ._("Started:", Times.format(app.getStartedTime()))
-        ._(
+        .__("Started:", Times.format(app.getStartedTime()))
+        .__(
             "Elapsed:",
             StringUtils.formatTime(Times.elapsed(app.getStartedTime(),
                 app.getFinishedTime())))
-        ._(
+        .__(
             "Tracking URL:",
             app.getTrackingUrl() == null
                 || app.getTrackingUrl().equals(UNAVAILABLE) ? null : root_url(app
@@ -240,31 +240,31 @@ public class AppBlock extends HtmlBlock {
         && webUiType.equals(YarnWebParams.RM_WEB_UI)) {
       LogAggregationStatus status = getLogAggregationStatus();
       if (status == null) {
-        overviewTable._("Log Aggregation Status:", "N/A");
+        overviewTable.__("Log Aggregation Status:", "N/A");
       } else if (status == LogAggregationStatus.DISABLED
           || status == LogAggregationStatus.NOT_START
           || status == LogAggregationStatus.SUCCEEDED) {
-        overviewTable._("Log Aggregation Status:", status.name());
+        overviewTable.__("Log Aggregation Status:", status.name());
       } else {
-        overviewTable._("Log Aggregation Status:",
+        overviewTable.__("Log Aggregation Status:",
             root_url("logaggregationstatus", app.getAppId()), status.name());
       }
       long timeout = appReport.getApplicationTimeouts()
           .get(ApplicationTimeoutType.LIFETIME).getRemainingTime();
       if (timeout < 0) {
-        overviewTable._("Application Timeout (Remaining Time):", "Unlimited");
+        overviewTable.__("Application Timeout (Remaining Time):", "Unlimited");
       } else {
-        overviewTable._("Application Timeout (Remaining Time):",
+        overviewTable.__("Application Timeout (Remaining Time):",
             String.format("%d seconds", timeout));
       }
     }
-    overviewTable._("Diagnostics:",
+    overviewTable.__("Diagnostics:",
         app.getDiagnosticsInfo() == null ? "" : app.getDiagnosticsInfo());
-    overviewTable._("Unmanaged Application:", app.isUnmanagedApp());
-    overviewTable._("Application Node Label expression:",
+    overviewTable.__("Unmanaged Application:", app.isUnmanagedApp());
+    overviewTable.__("Application Node Label expression:",
         app.getAppNodeLabelExpression() == null ? "<Not set>"
             : app.getAppNodeLabelExpression());
-    overviewTable._("AM container Node Label expression:",
+    overviewTable.__("AM container Node Label expression:",
         app.getAmNodeLabelExpression() == null ? "<Not set>"
             : app.getAmNodeLabelExpression());
   }
@@ -276,7 +276,7 @@ public class AppBlock extends HtmlBlock {
     TBODY<TABLE<Hamlet>> tbody =
         html.table("#attempts").thead().tr().th(".id", "Attempt ID")
           .th(".started", "Started").th(".node", "Node").th(".logs", "Logs")
-          ._()._().tbody();
+          .__().__().tbody();
 
     StringBuilder attemptsTableData = new StringBuilder("[\n");
     for (final ApplicationAttemptReport appAttemptReport : attempts) {
@@ -312,7 +312,7 @@ public class AppBlock extends HtmlBlock {
             "Failed to read the AM container of the application attempt "
                 + appAttemptReport.getApplicationAttemptId() + ".";
         LOG.error(message, e);
-        html.p()._(message)._();
+        html.p().__(message).__();
         return;
       }
       long startTime = 0L;
@@ -346,9 +346,9 @@ public class AppBlock extends HtmlBlock {
     }
     attemptsTableData.append("]");
     html.script().$type("text/javascript")
-      ._("var attemptsTableData=" + attemptsTableData)._();
+      .__("var attemptsTableData=" + attemptsTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 
   private String clarifyAppState(YarnApplicationState state) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index 7f42343..d836e64 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -44,9 +44,9 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -134,7 +134,7 @@ public class AppsBlock extends HtmlBlock {
     catch( Exception e) {
       String message = "Failed to read the applications.";
       LOG.error(message, e);
-      html.p()._(message)._();
+      html.p().__(message).__();
       return;
     }
     renderData(html);
@@ -147,7 +147,7 @@ public class AppsBlock extends HtmlBlock {
           .th(".queue", "Queue").th(".priority", "Application Priority")
           .th(".starttime", "StartTime").th(".finishtime", "FinishTime")
           .th(".state", "State").th(".finalstatus", "FinalStatus")
-          .th(".progress", "Progress").th(".ui", "Tracking UI")._()._().tbody();
+          .th(".progress", "Progress").th(".ui", "Tracking UI").__().__().tbody();
 
     StringBuilder appsTableData = new StringBuilder("[\n");
     for (ApplicationReport appReport : appReports) {
@@ -218,8 +218,8 @@ public class AppsBlock extends HtmlBlock {
     }
     appsTableData.append("]");
     html.script().$type("text/javascript")
-      ._("var appsTableData=" + appsTableData)._();
+      .__("var appsTableData=" + appsTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index 893e823..fa35a3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
@@ -86,7 +85,7 @@ public class ContainerBlock extends HtmlBlock {
     } catch (Exception e) {
       String message = "Failed to read the container " + containerid + ".";
       LOG.error(message, e);
-      html.p()._(message)._();
+      html.p().__(message).__();
       return;
     }
 
@@ -99,32 +98,32 @@ public class ContainerBlock extends HtmlBlock {
     setTitle(join("Container ", containerid));
 
     info("Container Overview")
-      ._(
+      .__(
         "Container State:",
         container.getContainerState() == null ? UNAVAILABLE : container
           .getContainerState())
-      ._("Exit Status:", container.getContainerExitStatus())
-      ._(
+      .__("Exit Status:", container.getContainerExitStatus())
+      .__(
         "Node:",
         container.getNodeHttpAddress() == null ? "#" : container
           .getNodeHttpAddress(),
         container.getNodeHttpAddress() == null ? "N/A" : container
           .getNodeHttpAddress())
-      ._("Priority:", container.getPriority())
-      ._("Started:", Times.format(container.getStartedTime()))
-      ._(
+      .__("Priority:", container.getPriority())
+      .__("Started:", Times.format(container.getStartedTime()))
+      .__(
         "Elapsed:",
         StringUtils.formatTime(Times.elapsed(container.getStartedTime(),
           container.getFinishedTime())))
-      ._(
+      .__(
         "Resource:",
         container.getAllocatedMB() + " Memory, "
             + container.getAllocatedVCores() + " VCores")
-      ._("Logs:", container.getLogUrl() == null ? "#" : container.getLogUrl(),
+      .__("Logs:", container.getLogUrl() == null ? "#" : container.getLogUrl(),
           container.getLogUrl() == null ? "N/A" : "Logs")
-      ._("Diagnostics:", container.getDiagnosticsInfo() == null ?
+      .__("Diagnostics:", container.getDiagnosticsInfo() == null ?
           "" : container.getDiagnosticsInfo());
 
-    html._(InfoBlock.class);
+    html.__(InfoBlock.class);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ErrorsAndWarningsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ErrorsAndWarningsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ErrorsAndWarningsBlock.java
index 56db66c..75f6fcd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ErrorsAndWarningsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ErrorsAndWarningsBlock.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
 import org.apache.hadoop.yarn.util.Times;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import java.util.ArrayList;
@@ -74,19 +74,19 @@ public class ErrorsAndWarningsBlock extends HtmlBlock {
     }
 
     if (!isAdmin) {
-      html.div().p()._("This page is for admins only.")._()._();
+      html.div().p().__("This page is for admins only.").__().__();
       return;
     }
 
     if (log instanceof Log4JLogger) {
-      html._(ErrorMetrics.class);
-      html._(WarningMetrics.class);
+      html.__(ErrorMetrics.class);
+      html.__(WarningMetrics.class);
       html.div().button().$onclick("reloadPage()").b("View data for the last ")
-        ._().select().$id("cutoff").option().$value("60")._("1 min")._()
-        .option().$value("300")._("5 min")._().option().$value("900")
-        ._("15 min")._().option().$value("3600")._("1 hour")._().option()
-        .$value("21600")._("6 hours")._().option().$value("43200")
-        ._("12 hours")._().option().$value("86400")._("24 hours")._()._()._();
+        .__().select().$id("cutoff").option().$value("60").__("1 min").__()
+        .option().$value("300").__("5 min").__().option().$value("900")
+        .__("15 min").__().option().$value("3600").__("1 hour").__().option()
+        .$value("21600").__("6 hours").__().option().$value("43200")
+        .__("12 hours").__().option().$value("86400").__("24 hours").__().__().__();
 
       String script = "function reloadPage() {"
           + " var timePeriod = $(\"#cutoff\").val();"
@@ -97,7 +97,7 @@ public class ErrorsAndWarningsBlock extends HtmlBlock {
           + "  $(element).parent().siblings('.toggle-content').fadeToggle();"
           + "}";
 
-      html.script().$type("text/javascript")._(script)._();
+      html.script().$type("text/javascript").__(script).__();
 
       html.style(".toggle-content { display: none; }");
 
@@ -110,7 +110,7 @@ public class ErrorsAndWarningsBlock extends HtmlBlock {
       Hamlet.TBODY<Hamlet.TABLE<Hamlet>> errorsTable =
           html.table("#messages").thead().tr().th(".message", "Message")
             .th(".type", "Type").th(".count", "Count")
-            .th(".lasttime", "Latest Message Time")._()._().tbody();
+            .th(".lasttime", "Latest Message Time").__().__().tbody();
 
       // cutoff has to be in seconds
       cutoff.add((Time.now() - cutoffPeriodSeconds * 1000) / 1000);
@@ -145,18 +145,18 @@ public class ErrorsAndWarningsBlock extends HtmlBlock {
               }
 
               cell.pre().a().$href("#").$onclick("toggleContent(this);")
-                .$style("white-space: pre")._(displayMessage)._()._().div()
-                .$class("toggle-content").pre()._(message)._()._()._();
+                .$style("white-space: pre").__(displayMessage).__().__().div()
+                .$class("toggle-content").pre().__(message).__().__().__();
             } else {
-              cell.pre()._(message)._()._();
+              cell.pre().__(message).__().__();
             }
             Log4jWarningErrorMetricsAppender.Element ele = entry.getValue();
             row.td(type).td(String.valueOf(ele.count))
-              .td(Times.format(ele.timestampSeconds * 1000))._();
+              .td(Times.format(ele.timestampSeconds * 1000)).__();
           }
         }
       }
-      errorsTable._()._();
+      errorsTable.__().__();
     }
   }
 
@@ -199,19 +199,19 @@ public class ErrorsAndWarningsBlock extends HtmlBlock {
             html.div().$class("metrics").$style("padding-bottom: 20px");
         div.h3(tableHeading).table("#metricsoverview").thead()
           .$class("ui-widget-header").tr().th().$class("ui-state-default")
-          ._("Last 1 minute")._().th().$class("ui-state-default")
-          ._("Last 5 minutes")._().th().$class("ui-state-default")
-          ._("Last 15 minutes")._().th().$class("ui-state-default")
-          ._("Last 1 hour")._().th().$class("ui-state-default")
-          ._("Last 6 hours")._().th().$class("ui-state-default")
-          ._("Last 12 hours")._().th().$class("ui-state-default")
-          ._("Last 24 hours")._()._()._().tbody().$class("ui-widget-content")
+          .__("Last 1 minute").__().th().$class("ui-state-default")
+          .__("Last 5 minutes").__().th().$class("ui-state-default")
+          .__("Last 15 minutes").__().th().$class("ui-state-default")
+          .__("Last 1 hour").__().th().$class("ui-state-default")
+          .__("Last 6 hours").__().th().$class("ui-state-default")
+          .__("Last 12 hours").__().th().$class("ui-state-default")
+          .__("Last 24 hours").__().__().__().tbody().$class("ui-widget-content")
           .tr().td(String.valueOf(values.get(0)))
           .td(String.valueOf(values.get(1))).td(String.valueOf(values.get(2)))
           .td(String.valueOf(values.get(3))).td(String.valueOf(values.get(4)))
           .td(String.valueOf(values.get(5))).td(String.valueOf(values.get(6)))
-          ._()._()._();
-        div._();
+          .__().__().__();
+        div.__();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.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/webapp/AllApplicationsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java
index d32b271..6425da6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllApplicationsPage.java
@@ -31,17 +31,17 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.BODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.BODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
 
 public class AllApplicationsPage extends NMView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setTitle("Applications running on this node");
     set(DATATABLES_ID, "applications");
@@ -88,22 +88,22 @@ public class AllApplicationsPage extends NMView {
             .table("#applications")
               .thead()
                 .tr()
-                  .td()._("ApplicationId")._()
-                  .td()._("ApplicationState")._()
-                ._()
-               ._()
+                  .td().__("ApplicationId").__()
+                  .td().__("ApplicationState").__()
+                .__()
+               .__()
                .tbody();
       for (Entry<ApplicationId, Application> entry : this.nmContext
           .getApplications().entrySet()) {
         AppInfo info = new AppInfo(entry.getValue());
         tableBody
           .tr()
-            .td().a(url("application", info.getId()), info.getId())._()
-            .td()._(info.getState())
-            ._()
-          ._();
+            .td().a(url("application", info.getId()), info.getId()).__()
+            .td().__(info.getState())
+            .__()
+          .__();
       }
-      tableBody._()._()._();
+      tableBody.__().__().__();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.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/webapp/AllContainersPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
index 24b8575..3fc6f3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/AllContainersPage.java
@@ -31,17 +31,17 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.ContainerInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.BODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.BODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
 
 public class AllContainersPage extends NMView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setTitle("All containers running on this node");
     set(DATATABLES_ID, "containers");
@@ -82,24 +82,24 @@ public class AllContainersPage extends NMView {
         .table("#containers")
           .thead()
             .tr()
-              .td()._("ContainerId")._()
-              .td()._("ContainerState")._()
-              .td()._("logs")._()
-            ._()
-          ._().tbody();
+              .td().__("ContainerId").__()
+              .td().__("ContainerState").__()
+              .td().__("logs").__()
+            .__()
+          .__().tbody();
       for (Entry<ContainerId, Container> entry : this.nmContext
           .getContainers().entrySet()) {
         ContainerInfo info = new ContainerInfo(this.nmContext, entry.getValue());
         tableBody
           .tr()
             .td().a(url("container", info.getId()), info.getId())
-            ._()
-            .td()._(info.getState())._()
+            .__()
+            .td().__(info.getState()).__()
             .td()
-                .a(url(info.getShortLogLink()), "logs")._()
-          ._();
+                .a(url(info.getShortLogLink()), "logs").__()
+          .__();
       }
-      tableBody._()._()._();
+      tableBody.__().__().__();
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.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/webapp/ApplicationPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
index 2783b18..00f80ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ApplicationPage.java
@@ -30,12 +30,11 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.AppInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -43,7 +42,7 @@ import com.google.inject.Inject;
 
 public class ApplicationPage extends NMView implements YarnWebParams {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     set(DATATABLES_ID, "containers");
@@ -80,30 +79,30 @@ public class ApplicationPage extends NMView implements YarnWebParams {
       try {
         applicationID = ApplicationId.fromString($(APPLICATION_ID));
       } catch (IllegalArgumentException e) {
-        html.p()._("Invalid Application Id " + $(APPLICATION_ID))._();
+        html.p().__("Invalid Application Id " + $(APPLICATION_ID)).__();
         return;
       }
       DIV<Hamlet> div = html.div("#content");
       Application app = this.nmContext.getApplications().get(applicationID);
       if (app == null) {
         div.h1("Unknown application with id " + applicationID
-            + ". Application might have been completed")._();
+            + ". Application might have been completed").__();
         return;
       }
       AppInfo info = new AppInfo(app);
       info("Application's information")
-            ._("ApplicationId", info.getId())
-            ._("ApplicationState", info.getState())
-            ._("User", info.getUser());
-      TABLE<Hamlet> containersListBody = html._(InfoBlock.class)
+            .__("ApplicationId", info.getId())
+            .__("ApplicationState", info.getState())
+            .__("User", info.getUser());
+      TABLE<Hamlet> containersListBody = html.__(InfoBlock.class)
           .table("#containers");
       for (String containerIdStr : info.getContainers()) {
         containersListBody
                .tr().td()
                  .a(url("container", containerIdStr), containerIdStr)
-                 ._()._();
+                 .__().__();
       }
-      containersListBody._();
+      containersListBody.__();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.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/webapp/ContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
index 3e5f4d2..f619e2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsPage.java
@@ -37,12 +37,11 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.PRE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.PRE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -51,7 +50,7 @@ public class ContainerLogsPage extends NMView {
   
   public static final String REDIRECT_URL = "redirect.url";
   
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String redirectUrl = $(REDIRECT_URL);
     if (redirectUrl == null || redirectUrl.isEmpty()) {
       set(TITLE, join("Logs for ", $(CONTAINER_ID)));
@@ -142,10 +141,10 @@ public class ContainerLogsPage extends NMView {
         try {
           long toRead = end - start;
           if (toRead < logFile.length()) {
-            html.p()._("Showing " + toRead + " bytes. Click ")
+            html.p().__("Showing " + toRead + " bytes. Click ")
                 .a(url("containerlogs", $(CONTAINER_ID), $(APP_OWNER), 
                     logFile.getName(), "?start=0"), "here").
-                    _(" for full log")._();
+                __(" for full log").__();
           }
           
           IOUtils.skipFully(logByteStream, start);
@@ -160,12 +159,12 @@ public class ContainerLogsPage extends NMView {
 
           while ((len = reader.read(cbuf, 0, currentToRead)) > 0
               && toRead > 0) {
-            pre._(new String(cbuf, 0, len));
+            pre.__(new String(cbuf, 0, len));
             toRead = toRead - len;
             currentToRead = toRead > bufferSize ? bufferSize : (int) toRead;
           }
 
-          pre._();
+          pre.__();
           reader.close();
 
         } catch (IOException e) {
@@ -199,7 +198,7 @@ public class ContainerLogsPage extends NMView {
                 .a(url("containerlogs", $(CONTAINER_ID), $(APP_OWNER),
                     logFile.getName(), "?start=-4096"),
                     logFile.getName() + " : Total file length is "
-                        + logFile.length() + " bytes.")._();
+                        + logFile.length() + " bytes.").__();
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.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/webapp/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
index 4beccc9..8117dca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerPage.java
@@ -25,11 +25,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.ContainerInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -38,7 +37,7 @@ import com.google.inject.Inject;
 public class ContainerPage extends NMView implements YarnWebParams {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     setTitle("Container " + $(CONTAINER_ID));
@@ -65,7 +64,7 @@ public class ContainerPage extends NMView implements YarnWebParams {
       try {
         containerID = ContainerId.fromString($(CONTAINER_ID));
       } catch (IllegalArgumentException e) {
-        html.p()._("Invalid containerId " + $(CONTAINER_ID))._();
+        html.p().__("Invalid containerId " + $(CONTAINER_ID)).__();
         return;
       }
 
@@ -73,22 +72,22 @@ public class ContainerPage extends NMView implements YarnWebParams {
       Container container = this.nmContext.getContainers().get(containerID);
       if (container == null) {
         div.h1("Unknown Container. Container might have completed, "
-                + "please go back to the previous page and retry.")._();
+                + "please go back to the previous page and retry.").__();
         return;
       }
       ContainerInfo info = new ContainerInfo(this.nmContext, container);
 
       info("Container information")
-        ._("ContainerID", info.getId())
-        ._("ContainerState", info.getState())
-        ._("ExitStatus", info.getExitStatus())
-        ._("Diagnostics", info.getDiagnostics())
-        ._("User", info.getUser())
-        ._("TotalMemoryNeeded", info.getMemoryNeeded())
-        ._("TotalVCoresNeeded", info.getVCoresNeeded())
-        ._("ExecutionType", info.getExecutionType())
-        ._("logs", info.getShortLogLink(), "Link to logs");
-      html._(InfoBlock.class);
+        .__("ContainerID", info.getId())
+        .__("ContainerState", info.getState())
+        .__("ExitStatus", info.getExitStatus())
+        .__("Diagnostics", info.getDiagnostics())
+        .__("User", info.getUser())
+        .__("TotalMemoryNeeded", info.getMemoryNeeded())
+        .__("TotalVCoresNeeded", info.getVCoresNeeded())
+        .__("ExecutionType", info.getExecutionType())
+        .__("logs", info.getShortLogLink(), "Link to logs");
+      html.__(InfoBlock.class);
     }
   }
 }


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
index 6403e3b..c5117ed 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
@@ -39,16 +39,15 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
-import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TFOOT;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.THEAD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
-import com.google.common.base.Joiner;
 import com.google.inject.Inject;
 
 /**
@@ -110,7 +109,7 @@ public class HsTaskPage extends HsView {
       headRow.th("Elapsed Time").
               th(".note", "Note");
       
-       TBODY<TABLE<Hamlet>> tbody = headRow._()._().tbody();
+       TBODY<TABLE<Hamlet>> tbody = headRow.__().__().tbody();
        // Write all the data into a JavaScript array of arrays for JQuery
        // DataTables to display
        StringBuilder attemptsTableData = new StringBuilder("[\n");
@@ -182,55 +181,55 @@ public class HsTaskPage extends HsView {
        }
        attemptsTableData.append("]");
        html.script().$type("text/javascript").
-       _("var attemptsTableData=" + attemptsTableData)._();
+           __("var attemptsTableData=" + attemptsTableData).__();
 
-      TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
+      TR<TFOOT<TABLE<Hamlet>>> footRow = tbody.__().tfoot().tr();
       footRow.
           th().input("search_init").$type(InputType.text).
-              $name("attempt_name").$value("Attempt")._()._().
+              $name("attempt_name").$value("Attempt").__().__().
           th().input("search_init").$type(InputType.text).
-              $name("attempt_state").$value("State")._()._().
+              $name("attempt_state").$value("State").__().__().
           th().input("search_init").$type(InputType.text).
-              $name("attempt_status").$value("Status")._()._().
+              $name("attempt_status").$value("Status").__().__().
           th().input("search_init").$type(InputType.text).
-              $name("attempt_node").$value("Node")._()._().
+              $name("attempt_node").$value("Node").__().__().
           th().input("search_init").$type(InputType.text).
-              $name("attempt_node").$value("Logs")._()._().
+              $name("attempt_node").$value("Logs").__().__().
           th().input("search_init").$type(InputType.text).
-              $name("attempt_start_time").$value("Start Time")._()._();
+              $name("attempt_start_time").$value("Start Time").__().__();
       
       if(type == TaskType.REDUCE) {
         footRow.
         th().input("search_init").$type(InputType.text).
-            $name("shuffle_time").$value("Shuffle Time")._()._();
+            $name("shuffle_time").$value("Shuffle Time").__().__();
         footRow.
         th().input("search_init").$type(InputType.text).
-            $name("merge_time").$value("Merge Time")._()._();
+            $name("merge_time").$value("Merge Time").__().__();
       }
       
       footRow.
         th().input("search_init").$type(InputType.text).
-            $name("attempt_finish").$value("Finish Time")._()._();
+            $name("attempt_finish").$value("Finish Time").__().__();
       
       if(type == TaskType.REDUCE) {
         footRow.
         th().input("search_init").$type(InputType.text).
-            $name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
+            $name("elapsed_shuffle_time").$value("Elapsed Shuffle Time").__().__();
         footRow.
         th().input("search_init").$type(InputType.text).
-            $name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
+            $name("elapsed_merge_time").$value("Elapsed Merge Time").__().__();
         footRow.
         th().input("search_init").$type(InputType.text).
-            $name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
+            $name("elapsed_reduce_time").$value("Elapsed Reduce Time").__().__();
       }
 
       footRow.
         th().input("search_init").$type(InputType.text).
-            $name("attempt_elapsed").$value("Elapsed Time")._()._().
+            $name("attempt_elapsed").$value("Elapsed Time").__().__().
         th().input("search_init").$type(InputType.text).
-            $name("note").$value("Note")._()._();
+            $name("note").$value("Note").__().__();
       
-      footRow._()._()._();
+      footRow.__().__().__();
     }
 
     protected String getAttemptId(TaskId taskId, TaskAttemptInfo ta) {
@@ -256,7 +255,7 @@ public class HsTaskPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     //override the nav config from commonPReHead
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
index 9511c06..702c13c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
@@ -28,14 +28,13 @@ import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
-import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TFOOT;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.THEAD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -74,10 +73,10 @@ public class HsTasksBlock extends HtmlBlock {
     //Create the spanning row
     int attemptColSpan = type == TaskType.REDUCE ? 8 : 3;
     thead.tr().
-      th().$colspan(5).$class("ui-state-default")._("Task")._().
+      th().$colspan(5).$class("ui-state-default").__("Task").__().
       th().$colspan(attemptColSpan).$class("ui-state-default").
-        _("Successful Attempt")._().
-    _();
+        __("Successful Attempt").__().
+        __();
 
     TR<THEAD<TABLE<Hamlet>>> theadRow = thead.
           tr().
@@ -102,7 +101,7 @@ public class HsTasksBlock extends HtmlBlock {
     }
     theadRow.th("Elapsed Time"); //Attempt
 
-    TBODY<TABLE<Hamlet>> tbody = theadRow._()._().tbody();
+    TBODY<TABLE<Hamlet>> tbody = theadRow.__().__().tbody();
 
     // Write all the data into a JavaScript array of arrays for JQuery
     // DataTables to display
@@ -173,41 +172,41 @@ public class HsTasksBlock extends HtmlBlock {
     }
     tasksTableData.append("]");
     html.script().$type("text/javascript").
-    _("var tasksTableData=" + tasksTableData)._();
+        __("var tasksTableData=" + tasksTableData).__();
     
-    TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
+    TR<TFOOT<TABLE<Hamlet>>> footRow = tbody.__().tfoot().tr();
     footRow.th().input("search_init").$type(InputType.text).$name("task")
-        .$value("ID")._()._().th().input("search_init").$type(InputType.text)
-        .$name("state").$value("State")._()._().th().input("search_init")
-        .$type(InputType.text).$name("start_time").$value("Start Time")._()._()
+        .$value("ID").__().__().th().input("search_init").$type(InputType.text)
+        .$name("state").$value("State").__().__().th().input("search_init")
+        .$type(InputType.text).$name("start_time").$value("Start Time").__().__()
         .th().input("search_init").$type(InputType.text).$name("finish_time")
-        .$value("Finish Time")._()._().th().input("search_init")
-        .$type(InputType.text).$name("elapsed_time").$value("Elapsed Time")._()
-        ._().th().input("search_init").$type(InputType.text)
-        .$name("attempt_start_time").$value("Start Time")._()._();
+        .$value("Finish Time").__().__().th().input("search_init")
+        .$type(InputType.text).$name("elapsed_time").$value("Elapsed Time").__()
+        .__().th().input("search_init").$type(InputType.text)
+        .$name("attempt_start_time").$value("Start Time").__().__();
 
     if(type == TaskType.REDUCE) {
       footRow.th().input("search_init").$type(InputType.text)
-          .$name("shuffle_time").$value("Shuffle Time")._()._();
+          .$name("shuffle_time").$value("Shuffle Time").__().__();
       footRow.th().input("search_init").$type(InputType.text)
-          .$name("merge_time").$value("Merge Time")._()._();
+          .$name("merge_time").$value("Merge Time").__().__();
     }
 
     footRow.th().input("search_init").$type(InputType.text)
-        .$name("attempt_finish").$value("Finish Time")._()._();
+        .$name("attempt_finish").$value("Finish Time").__().__();
 
     if(type == TaskType.REDUCE) {
       footRow.th().input("search_init").$type(InputType.text)
-          .$name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
+          .$name("elapsed_shuffle_time").$value("Elapsed Shuffle Time").__().__();
       footRow.th().input("search_init").$type(InputType.text)
-          .$name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
+          .$name("elapsed_merge_time").$value("Elapsed Merge Time").__().__();
       footRow.th().input("search_init").$type(InputType.text)
-          .$name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
+          .$name("elapsed_reduce_time").$value("Elapsed Reduce Time").__().__();
     }
 
     footRow.th().input("search_init").$type(InputType.text)
-        .$name("attempt_elapsed").$value("Elapsed Time")._()._();
+        .$name("attempt_elapsed").$value("Elapsed Time").__().__();
 
-    footRow._()._()._();
+    footRow.__().__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
index 3c3386e..d088591 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
@@ -41,7 +41,7 @@ public class HsTasksPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "tasks");
     set(DATATABLES_SELECTOR, ".dt-tasks" );

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
index 8e39087..510ece6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsView.java
@@ -39,7 +39,7 @@ public class HsView extends TwoColumnLayout {
    * (non-Javadoc)
    * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "jobs");
     set(initID(DATATABLES, "jobs"), jobsTableInit());
@@ -51,7 +51,7 @@ public class HsView extends TwoColumnLayout {
    * The prehead that should be common to all subclasses.
    * @param html used to render.
    */
-  protected void commonPreHead(Page.HTML<_> html) {
+  protected void commonPreHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index 505e20f..f17cf8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -341,4 +341,26 @@
       </plugin>
     </plugins>
   </build>
+  <!-- TODO: Remove this profile when hamlet was removed. -->
+  <profiles>
+    <profile>
+      <id>java9</id>
+      <activation>
+        <jdk>9</jdk>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <excludes>
+                <exclude>src/main/java/org/apache/hadoop/yarn/webapp/hamlet/**</exclude>
+                <exclude>src/main/test/org/apache/haodop/yarn/webapp/hamlet/**</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
index b04bc5d..94063ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
@@ -75,12 +75,12 @@ public class ResponseInfo implements Iterable<ResponseInfo.Item> {
     return about;
   }
 
-  public ResponseInfo _(String key, Object value) {
+  public ResponseInfo __(String key, Object value) {
     items.add(Item.of(key, value, false));
     return this;
   }
 
-  public ResponseInfo _(String key, String url, Object anchor) {
+  public ResponseInfo __(String key, String url, Object anchor) {
     if (url == null) {
       items.add(Item.of(key, anchor, false));
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/HelloWorld.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/HelloWorld.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/HelloWorld.java
index 9b5cbd1..4376b6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/HelloWorld.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/HelloWorld.java
@@ -41,11 +41,11 @@ public class HelloWorld {
   }
 
   public static class HelloView extends HtmlPage {
-    @Override protected void render(Page.HTML<_> html) {
+    @Override protected void render(Page.HTML<__> html) {
       html. // produces valid html 4.01 strict
         title($("title")).
         p("#hello-for-css").
-          _($("title"))._()._();
+          __($("title")).__().__();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/MyApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/MyApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/MyApp.java
index 3973f68..e13a883 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/MyApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/example/MyApp.java
@@ -60,12 +60,12 @@ public class MyApp {
   public static class MyView extends HtmlPage {
     // You can inject the app in views if needed.
     @Override
-    public void render(Page.HTML<_> html) {
+    public void render(Page.HTML<__> html) {
       html.
         title("My App").
         p("#content_id_for_css_styling").
-          _("You can have", $("anything"))._()._();
-      // Note, there is no _(); (to parent element) method at root level.
+          __("You can have", $("anything")).__().__();
+      // Note, there is no __(); (to parent element) method at root level.
       // and IDE provides instant feedback on what level you're on in
       // the auto-completion drop-downs.
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/Hamlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/Hamlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/Hamlet.java
index 58d9066..7213865 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/Hamlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/Hamlet.java
@@ -29,6 +29,10 @@ import java.util.EnumSet;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.yarn.webapp.SubView;
 
+/**
+ * @deprecated Use org.apache.hadoop.yarn.webapp.hamlet2 package instead.
+ */
+@Deprecated
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public class Hamlet extends HamletImpl implements HamletSpec._Html {
   public Hamlet(PrintWriter out, int nestLevel, boolean wasInline) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletGen.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletGen.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletGen.java
index 5acb3f3..8a2db8f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletGen.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletGen.java
@@ -43,7 +43,9 @@ import org.slf4j.LoggerFactory;
 /**
  * Generates a specific hamlet implementation class from a spec class
  * using a generic hamlet implementation class.
+ * @deprecated Use org.apache.hadoop.yarn.webapp.hamlet2 package instead.
  */
+@Deprecated
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public class HamletGen {
   static final Logger LOG = LoggerFactory.getLogger(HamletGen.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletImpl.java
index d792d31..289ad70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletImpl.java
@@ -43,7 +43,9 @@ import org.apache.hadoop.yarn.webapp.WebAppException;
  * optimized to use a thread-local element pool.
  *
  * Prints HTML as it builds. So the order is important.
+ * @deprecated Use org.apache.hadoop.yarn.webapp.hamlet2 package instead.
  */
+@Deprecated
 @InterfaceAudience.Private
 public class HamletImpl extends HamletSpec {
   private static final String INDENT_CHARS = "  ";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletSpec.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletSpec.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletSpec.java
index 081516c..e3bb6d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletSpec.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/HamletSpec.java
@@ -43,7 +43,9 @@ import org.apache.hadoop.yarn.webapp.SubView;
  * UPPERCASE, corresponding to an element definition in the DTD. $lowercase is
  * used as attribute builder methods to differentiate from element builder
  * methods.
+ * @deprecated Use org.apache.hadoop.yarn.webapp.hamlet2 package instead.
  */
+@Deprecated
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public class HamletSpec {
   // The enum values are lowercase for better compression,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/package-info.java
index 3286ab5..4d0cf49 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet/package-info.java
@@ -15,6 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+/**
+ * Deprecated.
+ * Use org.apache.hadoop.yarn.webapp.hamlet2 package instead.
+ */
+@Deprecated
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 package org.apache.hadoop.yarn.webapp.hamlet;
 import org.apache.hadoop.classification.InterfaceAudience;


---------------------------------------------------------------------
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-5548. Use MockRMMemoryStateStore to reduce test failures (Bibin A Chundatt via Varun Saxena)

Posted by in...@apache.org.
YARN-5548. Use MockRMMemoryStateStore to reduce test failures (Bibin A Chundatt via Varun Saxena)


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

Branch: refs/heads/HDFS-10467
Commit: f66fd11e514fb326fd1f37a88b444a5276f0947b
Parents: 27a1a5f
Author: Varun Saxena <va...@apache.org>
Authored: Thu Jul 27 23:14:50 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Thu Jul 27 23:14:50 2017 +0530

----------------------------------------------------------------------
 .../v2/app/rm/TestRMContainerAllocator.java     |   7 +-
 .../yarn/server/resourcemanager/MockRM.java     |  14 +-
 .../resourcemanager/TestApplicationCleanup.java |  30 +--
 .../TestContainerResourceUsage.java             |   8 +-
 .../TestNodeBlacklistingOnAMFailures.java       |   8 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |  12 +-
 .../server/resourcemanager/TestRMRestart.java   | 228 ++++++++-----------
 .../TestWorkPreservingRMRestart.java            | 117 ++++------
 .../applicationsmanager/TestAMRestart.java      |  29 +--
 .../rmapp/TestApplicationLifetimeMonitor.java   |   7 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   7 +-
 .../capacity/TestApplicationPriority.java       |  18 +-
 .../capacity/TestCapacityScheduler.java         |  23 +-
 ...TestWorkPreservingRMRestartForNodeLabel.java |  10 +-
 .../scheduler/fair/TestFairScheduler.java       |   8 +-
 .../security/TestDelegationTokenRenewer.java    |   5 +-
 .../security/TestRMDelegationTokens.java        |   9 +-
 17 files changed, 199 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index 8879362..bc05c62 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -2414,10 +2414,7 @@ public class TestRMContainerAllocator {
     conf.setInt(
         MRJobConfig.MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, -1);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MyResourceManager rm1 = new MyResourceManager(conf, memStore);
+    MyResourceManager rm1 = new MyResourceManager(conf);
     rm1.start();
 
     // Submit the application
@@ -2504,7 +2501,7 @@ public class TestRMContainerAllocator {
     assertBlacklistAdditionsAndRemovals(0, 0, rm1);
 
     // Phase-2 start 2nd RM is up
-    MyResourceManager rm2 = new MyResourceManager(conf, memStore);
+    MyResourceManager rm2 = new MyResourceManager(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     allocator.updateSchedulerProxy(rm2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/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 5a215e5..e967807 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
@@ -102,8 +102,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 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;
-
-
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
 import org.apache.log4j.Level;
@@ -174,14 +172,6 @@ public class MockRM extends ResourceManager {
     disableDrainEventsImplicitly = false;
   }
 
-  public class MockRMMemoryStateStore extends MemoryRMStateStore {
-    @SuppressWarnings("rawtypes")
-    @Override
-    protected EventHandler getRMStateStoreEventHandler() {
-      return rmStateStoreEventHandler;
-    }
-  }
-
   public class MockRMNullStateStore extends NullRMStateStore {
     @SuppressWarnings("rawtypes")
     @Override
@@ -1294,4 +1284,8 @@ public class MockRM extends ResourceManager {
       ((AsyncDispatcher) getRmDispatcher()).disableExitOnDispatchException();
     }
   }
+
+  public RMStateStore getRMStateStore() {
+    return getRMContext().getStateStore();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.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/TestApplicationCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
index 422b7eb..ebca7a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
@@ -153,7 +153,6 @@ public class TestApplicationCleanup {
     rm.stop();
   }
 
-  @SuppressWarnings("resource")
   @Test
   public void testContainerCleanup() throws Exception {
 
@@ -291,11 +290,8 @@ public class TestApplicationCleanup {
   @Test (timeout = 60000)
   public void testAppCleanupWhenRMRestartedAfterAppFinished() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -308,7 +304,7 @@ public class TestApplicationCleanup {
     rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
 
     // start new RM
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     
     // nm1 register to rm2, and do a heartbeat
@@ -327,11 +323,9 @@ public class TestApplicationCleanup {
   @Test(timeout = 60000)
   public void testAppCleanupWhenRMRestartedBeforeAppFinished() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 1024, rm1.getResourceTrackerService());
@@ -357,7 +351,7 @@ public class TestApplicationCleanup {
     }
 
     // start new RM
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
 
     // nm1/nm2 register to rm2, and do a heartbeat
@@ -383,16 +377,12 @@ public class TestApplicationCleanup {
     rm2.stop();
   }
 
-  @SuppressWarnings("resource")
   @Test (timeout = 60000)
   public void testContainerCleanupWhenRMRestartedAppNotRegistered() throws
       Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -405,7 +395,7 @@ public class TestApplicationCleanup {
     rm1.waitForState(app0.getApplicationId(), RMAppState.RUNNING);
 
     // start new RM
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
 
     // nm1 register to rm2, and do a heartbeat
@@ -426,11 +416,9 @@ public class TestApplicationCleanup {
   @Test (timeout = 60000)
   public void testAppCleanupWhenNMReconnects() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -466,11 +454,9 @@ public class TestApplicationCleanup {
   @Test(timeout = 60000)
   public void testProcessingNMContainerStatusesOnNMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // 1. Start the cluster-RM,NM,Submit app with 1024MB,Launch & register AM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     int nmMemory = 8192;
     int amMemory = 1024;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.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/TestContainerResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
index 3db00a2..ba9de6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
@@ -138,10 +138,8 @@ public class TestContainerResourceUsage {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MockRM rm0 = new MockRM(conf, memStore);
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm0 = new MockRM(conf);
     rm0.start();
     MockNM nm =
         new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
@@ -229,7 +227,7 @@ public class TestContainerResourceUsage {
         vcoreSeconds, metricsBefore.getVcoreSeconds());
 
     // create new RM to represent RM restart. Load up the state store.
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf, rm0.getRMStateStore());
     rm1.start();
     RMApp app0After =
         rm1.getRMContext().getRMApps().get(app0.getApplicationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.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/TestNodeBlacklistingOnAMFailures.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
index 75ef5c7..5266210 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestNodeBlacklistingOnAMFailures.java
@@ -413,11 +413,9 @@ public class TestNodeBlacklistingOnAMFailures {
   }
 
   private MockRM startRM(YarnConfiguration conf) {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MockRM rm = new MockRM(conf, memStore);
-
+    conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm = new MockRM(conf);
     rm.start();
     return rm;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.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/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
index ec6b1e6..b5293a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
@@ -415,7 +415,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
       int count = 0;
 
       @Override
@@ -465,7 +465,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
       @Override
       public void updateApplicationState(ApplicationStateData appState) {
         notifyStoreOperationFailed(new StoreFencedException());
@@ -530,12 +530,10 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     Configuration conf = new YarnConfiguration(configuration);
-
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
 
     // 1. start RM
-    rm = new MockRM(conf, memStore);
+    rm = new MockRM(conf);
     rm.init(conf);
     rm.start();
 
@@ -565,7 +563,7 @@ public class TestRMHA {
     verifyClusterMetrics(0, 0, 0, 0, 0, 0);
 
     // 3. Create new RM
-    rm = new MockRM(conf, memStore) {
+    rm = new MockRM(conf, rm.getRMStateStore()) {
       @Override
       protected ResourceTrackerService createResourceTrackerService() {
         return new ResourceTrackerService(this.rmContext,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 955b4b6..f9d0eae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -178,24 +178,23 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     return rm;
   }
 
-  @SuppressWarnings("rawtypes")
+  private MockRM createMockRM(YarnConfiguration config) {
+    MockRM rm = new MockRM(config);
+    rms.add(rm);
+    return rm;
+  }
+
   @Test (timeout=180000)
   public void testRMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+    // PHASE 1: create RM and get state
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-                                                  rmState.getApplicationState();
-    
-    
-    // PHASE 1: create state in an RM
-    
-    // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
-    
+        memStore.getState().getApplicationState();
+
     // start like normal because state is empty
     rm1.start();
     
@@ -451,14 +450,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   public void testRMRestartAppRunningAMFailed() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-    Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
 
-    // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    // Create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    Map<ApplicationId, ApplicationStateData> rmAppState =
+        memStore.getState().getApplicationState();
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -508,14 +505,13 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     // be started immediately.
     YarnConfiguration conf = new YarnConfiguration(this.conf);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 40);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-    
+        memStore.getState().getApplicationState();
     // start RM
-    final MockRM rm1 = createMockRM(conf, memStore);
     rm1.start();
     AbstractYarnScheduler ys =
         (AbstractYarnScheduler)rm1.getResourceScheduler();
@@ -674,7 +670,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
       int count = 0;
 
       @Override
@@ -727,14 +723,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   public void testRMRestartFailedApp() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-
+        memStore.getState().getApplicationState();
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -775,14 +769,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   public void testRMRestartKilledApp() throws Exception{
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-
+        memStore.getState().getApplicationState();
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -823,7 +815,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 60000)
   public void testRMRestartKilledAppWithNoAttempts() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
       @Override
       public synchronized void storeApplicationAttemptStateInternal(
           ApplicationAttemptId attemptId,
@@ -865,14 +857,13 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   public void testRMRestartSucceededApp() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+    // PHASE 1: create RM and get state
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
+        memStore.getState().getApplicationState();
 
-    // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    // start like normal because state is empty
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -913,11 +904,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   public void testRMRestartGetApplicationList() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore) {
+    MockRM rm1 = new MockRM(conf) {
       @Override
       protected SystemMetricsPublisher createSystemMetricsPublisher() {
         return spy(super.createSystemMetricsPublisher());
@@ -956,7 +944,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     .appCreated(any(RMApp.class), anyLong());
     // restart rm
 
-    MockRM rm2 = new MockRM(conf, memStore) {
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
       @Override
       protected RMAppManager createRMAppManager() {
         return spy(super.createRMAppManager());
@@ -1077,13 +1065,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();  
-    MockRM rm1 = createMockRM(conf, memStore);
+        memStore.getState().getApplicationState();
+    // start RM
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1146,16 +1133,15 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   public void testRMRestartTimelineCollectorContext() throws Exception {
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-    Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
+
     MockRM rm1 = null;
     MockRM rm2 = null;
     try {
-      rm1 = createMockRM(conf, memStore);
+      rm1 = createMockRM(conf);
       rm1.start();
+      MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+      Map<ApplicationId, ApplicationStateData> rmAppState =
+          memStore.getState().getApplicationState();
       MockNM nm1 =
           new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
       nm1.registerNode();
@@ -1212,13 +1198,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         "kerberos");
     UserGroupInformation.setConfiguration(conf);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-
+    // create RM
+    MockRM rm1 = new TestSecurityMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
+        memStore.getState().getApplicationState();
+    // start RM
     rm1.start();
 
     HashSet<Token<RMDelegationTokenIdentifier>> tokenSet =
@@ -1307,13 +1292,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       "kerberos");
     UserGroupInformation.setConfiguration(conf);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-
+    // create RM
+    MockRM rm1 = new TestSecurityMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
+        memStore.getState().getApplicationState();
+    // start RM
     rm1.start();
     MockNM nm1 =
         new MockNM("0.0.0.0:4321", 15120, rm1.getResourceTrackerService());
@@ -1388,8 +1372,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         "kerberos");
     conf.set(YarnConfiguration.RM_ADDRESS, "localhost:8032");
     UserGroupInformation.setConfiguration(conf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
+
+    MockRM rm1 = new TestSecurityMockRM(conf);
+    rm1.start();
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     RMState rmState = memStore.getState();
 
     Map<ApplicationId, ApplicationStateData> rmAppState =
@@ -1399,10 +1385,6 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     Set<DelegationKey> rmDTMasterKeyState =
         rmState.getRMDTSecretManagerState().getMasterKeyState();
 
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
-
-    rm1.start();
-
     // create an empty credential
     Credentials ts = new Credentials();
 
@@ -1537,10 +1519,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         "kerberos");
     conf.set(YarnConfiguration.RM_ADDRESS, "localhost:8032");
     UserGroupInformation.setConfiguration(conf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
+    MockRM rm1 = new TestSecurityMockRM(conf);
     rm1.start();
 
     GetDelegationTokenRequest request1 =
@@ -1553,7 +1533,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         ConverterUtils.convertFromYarn(response1.getRMDelegationToken(), rmAddr);
 
     // start new RM
-    MockRM rm2 = new TestSecurityMockRM(conf, memStore);
+    MockRM rm2 = new TestSecurityMockRM(conf, rm1.getRMStateStore());
     rm2.start();
 
     // submit an app with the old delegation token got from previous RM.
@@ -1631,14 +1611,13 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 60000)
   public void testFinishedAppRemovalAfterRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
     conf.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 1);
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
 
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    MockRM rm1 = createMockRM(conf);
     rm1.start();
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    RMState rmState = memStore.getState();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -1676,7 +1655,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   // This is to test RM does not get hang on shutdown.
   @Test (timeout = 10000)
   public void testRMShutdown() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
       @Override
       public synchronized void checkVersion()
           throws Exception {
@@ -1742,10 +1721,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
       "kerberos");
     UserGroupInformation.setConfiguration(conf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore) {
+    MockRM rm1 = new TestSecurityMockRM(conf) {
       class TestDelegationTokenRenewer extends DelegationTokenRenewer {
         public void addApplicationAsync(ApplicationId applicationId, Credentials ts,
             boolean shouldCancelAtEnd, String user, Configuration appConf) {
@@ -1758,6 +1734,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     };
     rm1.start();
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     RMApp app1 = null;
     try {
        app1 = rm1.submitApp(200, "name", "user",
@@ -1781,7 +1758,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 20000)
   public void testAppRecoveredInOrderOnRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
     memStore.init(conf);
 
     for (int i = 10; i > 0; i--) {
@@ -1836,12 +1813,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   public void testQueueMetricsOnRMRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    // PHASE 1: create state in an RM
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    MockRM rm1 = createMockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1879,7 +1852,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
     // PHASE 2: create new RM and start from old state
     // create new RM to represent restart and recover state
-    MockRM rm2 = createMockRM(conf, memStore);
+    MockRM rm2 = createMockRM(conf, rm1.getRMStateStore());
     QueueMetrics qm2 = rm2.getResourceScheduler().getRootQueueMetrics();
     resetQueueMetrics(qm2);
     assertQueueMetrics(qm2, 0, 0, 0, 0);
@@ -1960,7 +1933,6 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 60000)
   public void testDecomissionedNMsMetricsOnRMRestart() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
     conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
       hostFile.getAbsolutePath());
     writeToHostsFile("");
@@ -2039,11 +2011,9 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
       "kerberos");
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
+    MockRM rm1 = createMockRM(conf);
     rm1.start();
     final MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2051,7 +2021,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     RMApp app0 = rm1.submitApp(200);
     final MockAM am0 = MockRM.launchAndRegisterAM(app0, rm1, nm1);
 
-    MockRM rm2 = new MockRM(conf, memStore) {
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
       @Override
       protected ResourceTrackerService createResourceTrackerService() {
         return new ResourceTrackerService(this.rmContext,
@@ -2158,6 +2128,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       super(conf, store);
     }
 
+    public TestSecurityMockRM(Configuration conf) {
+      super(conf);
+    }
+
     @Override
     public void init(Configuration conf) {
       // reset localServiceAddress.
@@ -2208,10 +2182,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.set(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
         nodeLabelFsStoreDirURI);
     
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
-    MockRM rm1 = new MockRM(conf, memStore) {
+    MockRM rm1 = new MockRM(conf) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();
@@ -2261,7 +2233,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     Assert.assertEquals(1, nodeLabelManager.getNodeLabels().size());
     Assert.assertTrue(nodeLabels.get(n1).equals(toSet("y")));
 
-    MockRM rm2 = new MockRM(conf, memStore) {
+    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();
@@ -2290,14 +2262,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     int maxAttempt =
         conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
             YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
-        rmState.getApplicationState();
-
+        memStore.getState().getApplicationState();
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2365,10 +2335,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.set(YarnConfiguration.FS_NODE_LABELS_STORE_ROOT_DIR,
         nodeLabelFsStoreDirURI);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
-    MockRM rm1 = new MockRM(conf, memStore) {
+    MockRM rm1 = new MockRM(conf) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();
@@ -2396,7 +2364,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     nodeLabelManager.replaceLabelsOnNode(ImmutableMap.of(n1, toSet("x")));
     MockRM rm2 = null;
     for (int i = 0; i < 2; i++) {
-      rm2 = new MockRM(conf, memStore) {
+      rm2 = new MockRM(conf, rm1.getRMStateStore()) {
         @Override
         protected RMNodeLabelsManager createNodeLabelManager() {
           RMNodeLabelsManager mgr = new RMNodeLabelsManager();
@@ -2419,15 +2387,12 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test(timeout = 120000)
   public void testRMRestartAfterPreemption() throws Exception {
-    Configuration conf = new Configuration();
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     if (!getSchedulerType().equals(SchedulerType.CAPACITY)) {
       return;
     }
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
 
@@ -2466,7 +2431,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm2 = null;
     // start RM2
     try {
-      rm2 = new MockRM(conf, memStore);
+      rm2 = new MockRM(conf, rm1.getRMStateStore());
       rm2.start();
       Assert.assertTrue("RM start successfully", true);
     } catch (Exception e) {
@@ -2480,11 +2445,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test(timeout = 60000)
   public void testRMRestartOnMissingAttempts() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 5);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
+    // create RM
+    MockRM rm1 = createMockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     // start RM
-    MockRM rm1 = createMockRM(conf, memStore);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2540,13 +2504,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test(timeout = 60000)
   public void testRMRestartAfterNodeLabelDisabled() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
 
     MockRM rm1 = new MockRM(
-        TestUtils.getConfigurationWithDefaultQueueLabels(conf), memStore) {
+        TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();
@@ -2580,7 +2541,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     // restart rm with node label disabled
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, false);
     MockRM rm2 = new MockRM(
-        TestUtils.getConfigurationWithDefaultQueueLabels(conf), memStore) {
+        TestUtils.getConfigurationWithDefaultQueueLabels(conf),
+        rm1.getRMStateStore()) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.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/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index c4cf256..d028227 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -148,9 +148,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     int containerMemory = 1024;
     Resource containerResource = Resource.newInstance(containerMemory, 1);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -162,7 +160,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app1);
 
     // Re-start RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     // recover app
@@ -296,9 +294,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     int containerMemory = 1024;
     Resource containerResource = Resource.newInstance(containerMemory, 1);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(schedulerConf);
-    rm1 = new MockRM(schedulerConf, memStore);
+    rm1 = new MockRM(schedulerConf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -316,7 +312,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app1);
 
     // 3. Fail over (restart) RM.
-    rm2 = new MockRM(schedulerConf, memStore);
+    rm2 = new MockRM(schedulerConf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     // 4. Validate app is recovered post failover.
@@ -570,9 +566,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   public void testRMRestartWithRemovedQueue() throws Exception{
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "");
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -585,7 +579,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST});
     final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST;
     csConf.setCapacity(noQueue, 100);
-    rm2 = new MockRM(csConf,memStore);
+    rm2 = new MockRM(csConf, rm1.getRMStateStore());
 
     rm2.start();
     UserGroupInformation user2 = UserGroupInformation.createRemoteUser("user2");
@@ -622,9 +616,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
     setupQueueConfiguration(csConf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(csConf);
-    rm1 = new MockRM(csConf, memStore);
+    rm1 = new MockRM(csConf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -648,7 +640,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     csConf.set(PREFIX + "root.Default.QueueB.state", "STOPPED");
 
     // Re-start RM
-    rm2 = new MockRM(csConf, memStore);
+    rm2 = new MockRM(csConf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm2.setResourceTrackerService(rm2.getResourceTrackerService());
@@ -783,9 +775,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
     setupQueueConfiguration(csConf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(csConf);
-    rm1 = new MockRM(csConf, memStore);
+    rm1 = new MockRM(csConf);
     rm1.start();
     MockNM nm =
         new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
@@ -798,7 +788,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         getYarnApplicationState(), YarnApplicationState.RUNNING);
 
     // Take a copy of state store so that it can be reset to this state.
-    RMState state = memStore.loadState();
+    RMState state = rm1.getRMStateStore().loadState();
 
     // Change scheduler config with child queues added to QueueB.
     csConf = new CapacitySchedulerConfiguration(conf);
@@ -806,7 +796,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which is no longer a leaf queue after restart.";
-    verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags, memStore, state);
+    verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags,
+        (MemoryRMStateStore) rm1.getRMStateStore(), state);
   }
 
   //Test behavior of an app if queue is removed during recovery. Test case does
@@ -829,9 +820,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration(conf);
     setupQueueConfiguration(csConf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(csConf);
-    rm1 = new MockRM(csConf, memStore);
+    rm1 = new MockRM(csConf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -860,7 +849,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app2);
 
     // Take a copy of state store so that it can be reset to this state.
-    RMState state = memStore.loadState();
+    RMState state = rm1.getRMStateStore().loadState();
 
     // Set new configuration with QueueB removed.
     csConf = new CapacitySchedulerConfiguration(conf);
@@ -868,7 +857,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which no longer exists after restart.";
-    verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags, memStore, state);
+    verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags,
+        (MemoryRMStateStore) rm1.getRMStateStore(), state);
   }
 
   private void checkParentQueue(ParentQueue parentQueue, int numContainers,
@@ -883,10 +873,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // should not recover the containers that belong to the failed AM.
   @Test(timeout = 20000)
   public void testAMfailedBetweenRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
     conf.setLong(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -894,7 +882,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     RMApp app1 = rm1.submitApp(200);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
 
@@ -937,9 +925,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // recover containers for completed apps.
   @Test(timeout = 20000)
   public void testContainersNotRecoveredForCompletedApps() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -948,7 +934,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
 
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus runningContainer =
@@ -975,11 +961,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   @Test (timeout = 600000)
   public void testAppReregisterOnRMWorkPreservingRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // start RM
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -993,7 +977,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     am0.registerAppAttempt();
 
     // start new RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
@@ -1008,9 +992,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   
   @Test (timeout = 30000)
   public void testAMContainerStatusWithRMRestart() throws Exception {  
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -1025,7 +1007,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         attempt0.getMasterContainer().getId()).isAMContainer());
 
     // Re-start RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
 
@@ -1044,9 +1026,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   @Test (timeout = 20000)
   public void testRecoverSchedulerAppAndAttemptSynchronously() throws Exception {
     // start RM
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1056,7 +1036,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     RMApp app0 = rm1.submitApp(200);
     MockAM am0 = MockRM.launchAndRegisterAM(app0, rm1, nm1);
 
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     // scheduler app/attempt is immediately available after RM is re-started.
@@ -1077,9 +1057,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // container should not be recovered.
   @Test (timeout = 50000)
   public void testReleasedContainerNotRecovered() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     MockNM nm1 = new MockNM("h1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
     rm1.start();
@@ -1089,7 +1067,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
     // Re-start RM
     conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 8000);
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
@@ -1175,9 +1153,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
       throws Exception {
     conf.setLong(
       YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 4000);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -1186,7 +1162,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
     // Restart RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.registerNode();
@@ -1229,11 +1205,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   public void testRetriedFinishApplicationMasterRequest()
       throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
     // start RM
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1253,7 +1226,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
 
     // start new RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
 
     am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
@@ -1266,9 +1239,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
       "kerberos");
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     UserGroupInformation.setConfiguration(conf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new TestSecurityMockRM(conf, memStore);
+    MockRM rm1 = new TestSecurityMockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -1276,7 +1247,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     RMApp app1 = rm1.submitApp(200);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
 
-    MockRM rm2 = new TestSecurityMockRM(conf, memStore) {
+    MockRM rm2 = new TestSecurityMockRM(conf, rm1.getRMStateStore()) {
       protected DelegationTokenRenewer createDelegationTokenRenewer() {
         return new DelegationTokenRenewer() {
           @Override
@@ -1313,9 +1284,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
    */
   @Test (timeout = 30000)
   public void testAppFailToValidateResourceRequestOnRecovery() throws Exception{
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -1328,16 +1297,14 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 50);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 100);
 
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     rm2.start();
   }
 
   @Test(timeout = 20000)
   public void testContainerCompleteMsgNotLostAfterAMFailedAndRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
 
     MockNM nm1 =
@@ -1370,7 +1337,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app0, rm1, nm1);
 
     // rm failover
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
 
@@ -1439,11 +1406,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   @Test(timeout = 600000)
   public void testUAMRecoveryOnRMWorkPreservingRestart() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
     // start RM
-    rm1 = new MockRM(conf, memStore);
+    rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1471,7 +1436,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     Assert.assertFalse(conts.isEmpty());
 
     // start new RM
-    rm2 = new MockRM(conf, memStore);
+    rm2 = new MockRM(conf, rm1.getRMStateStore());
     rm2.start();
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
@@ -1521,7 +1486,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         recoveredApp.getFinalApplicationStatus());
 
     // Restart RM once more to check UAM is not re-run
-    MockRM rm3 = new MockRM(conf, memStore);
+    MockRM rm3 = new MockRM(conf, rm1.getRMStateStore());
     rm3.start();
     recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
     Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index f178884..528afac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -47,7 +47,6 @@ 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.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
@@ -381,9 +380,7 @@ public class TestAMRestart {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -405,7 +402,9 @@ public class TestAMRestart {
     Assert.assertTrue(! attempt1.shouldCountTowardsMaxAttemptRetry());
     rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
     ApplicationStateData appState =
-        memStore.getState().getApplicationState().get(app1.getApplicationId());
+        ((MemoryRMStateStore) rm1.getRMStateStore()).getState()
+            .getApplicationState().get(app1.getApplicationId());
+
     // AM should be restarted even though max-am-attempt is 1.
     MockAM am2 =
         rm1.waitForNewAMToLaunchAndRegister(app1.getApplicationId(), 2, nm1);
@@ -508,9 +507,7 @@ public class TestAMRestart {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -548,10 +545,9 @@ public class TestAMRestart {
 
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -630,10 +626,9 @@ public class TestAMRestart {
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
 
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     CapacityScheduler scheduler =
         (CapacityScheduler) rm1.getResourceScheduler();
@@ -706,10 +701,8 @@ public class TestAMRestart {
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.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/rmapp/TestApplicationLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
index fdc47b9..f7e76bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
@@ -164,9 +164,8 @@ public class TestApplicationLifetimeMonitor {
         true);
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
@@ -235,8 +234,6 @@ public class TestApplicationLifetimeMonitor {
       throws Exception {
     MockRM rm1 = null;
     try {
-      conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-
       MemoryRMStateStore memStore = new MemoryRMStateStore() {
         private int count = 0;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.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/TestAbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
index f1adb5e..60b9e4b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestAbstractYarnScheduler.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@@ -382,10 +381,8 @@ public class TestAbstractYarnScheduler extends ParameterizedSchedulerTestBase {
   @Test(timeout = 10000)
   public void testReleasedContainerIfAppAttemptisNull() throws Exception {
     YarnConfiguration conf=getConf();
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    MockRM rm1 = new MockRM(conf);
     try {
       rm1.start();
       MockNM nm1 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationPriority.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/TestApplicationPriority.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/TestApplicationPriority.java
index fd17bd9..cad0151 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/TestApplicationPriority.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/TestApplicationPriority.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -42,8 +41,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 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.attempt.RMAppAttemptImpl;
@@ -405,16 +402,11 @@ public class TestApplicationPriority {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, 10);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    RMState rmState = memStore.getState();
-    Map<ApplicationId, ApplicationStateData> rmAppState = rmState
-        .getApplicationState();
-
     // PHASE 1: create state in an RM
 
     // start RM
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
 
     MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
@@ -611,10 +603,8 @@ public class TestApplicationPriority {
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, 10);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
 
     MockNM nm1 =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/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 41a7ce8..0642cd9 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,7 +106,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMW
 import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
 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.recovery.MemoryRMStateStore;
 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;
@@ -2965,9 +2964,7 @@ public class TestCapacityScheduler {
         new YarnConfiguration(new CapacitySchedulerConfiguration());
     conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm = new MockRM(conf, memStore);
+    MockRM rm = new MockRM(conf);
     rm.start();
 
     HashMap<NodeId, MockNM> nodes = new HashMap<>();
@@ -3129,10 +3126,7 @@ public class TestCapacityScheduler {
         new YarnConfiguration(
             setupQueueConfiguration(new CapacitySchedulerConfiguration()));
     conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
-
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 100 * GB, rm1.getResourceTrackerService());
@@ -3212,9 +3206,7 @@ public class TestCapacityScheduler {
     YarnConfiguration conf = new YarnConfiguration(csConf);
     conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm1 = new MockRM(conf, memStore);
+    MockRM rm1 = new MockRM(conf);
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 24 * GB, rm1.getResourceTrackerService());
@@ -3259,9 +3251,7 @@ public class TestCapacityScheduler {
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
     mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm = new MockRM(conf, memStore) {
+    MockRM rm = new MockRM(conf) {
       protected RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
       }
@@ -3668,9 +3658,8 @@ public class TestCapacityScheduler {
     final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
     mgr.init(conf);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    MockRM rm = new MockRM(conf, memStore) {
+
+    MockRM rm = new MockRM(conf) {
       protected RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestWorkPreservingRMRestartForNodeLabel.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/TestWorkPreservingRMRestartForNodeLabel.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/TestWorkPreservingRMRestartForNodeLabel.java
index 0386aab..36ee68e 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/TestWorkPreservingRMRestartForNodeLabel.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/TestWorkPreservingRMRestartForNodeLabel.java
@@ -133,21 +133,17 @@ public class TestWorkPreservingRMRestartForNodeLabel {
     mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
         NodeId.newInstance("h2", 0), toSet("y")));
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-    
     conf = TestUtils.getConfigurationWithDefaultQueueLabels(conf);
-    
+
     // inject node label manager
     MockRM rm1 =
-        new MockRM(conf,
-            memStore) {
+        new MockRM(conf) {
           @Override
           public RMNodeLabelsManager createNodeLabelManager() {
             return mgr;
           }
         };
-
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.getRMContext().setNodeLabelManager(mgr);
     rm1.start();
     MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 2233287..0d54c33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -5112,12 +5112,10 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     out.println("</allocations>");
     out.close();
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(conf);
-
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // 3. start a active RM
-    MockRM rm2 = new MockRM(conf, memStore);
-    rm2.init(conf);
+    MockRM rm2 = new MockRM(conf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm2.getRMStateStore();
     rm2.start();
 
     MockNM nm =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.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/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index 0190db6..9fb9d42 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -1006,9 +1006,8 @@ public class TestDelegationTokenRenewer {
     Credentials credentials = new Credentials();
     credentials.addToken(userText1, originalToken);
 
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
-    memStore.init(yarnConf);
-    MockRM rm1 = new TestSecurityMockRM(yarnConf, memStore);
+    MockRM rm1 = new TestSecurityMockRM(yarnConf);
+    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     RMApp app = rm1.submitApp(200, "name", "user",
         new HashMap<ApplicationAccessType, String>(), false, "default", 1,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f66fd11e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.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/security/TestRMDelegationTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
index 80310a5..06c642a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRMMemoryStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
@@ -61,10 +62,10 @@ public class TestRMDelegationTokens {
     rootLogger.setLevel(Level.DEBUG);
     ExitUtil.disableSystemExit();
     testConf = new YarnConfiguration();
+    testConf
+        .set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     UserGroupInformation.setLoginUser(null);
     UserGroupInformation.setConfiguration(testConf);
-    testConf.set(YarnConfiguration.RM_STORE,
-        MemoryRMStateStore.class.getName());
   }
 
   // Test the DT mast key in the state-store when the mast key is being rolled.
@@ -73,7 +74,7 @@ public class TestRMDelegationTokens {
     Configuration conf = new Configuration(testConf);
     conf.set("hadoop.security.authentication", "kerberos");
     UserGroupInformation.setConfiguration(conf);
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
     memStore.init(conf);
     RMState rmState = memStore.getState();
 
@@ -127,7 +128,7 @@ public class TestRMDelegationTokens {
   // Test all expired keys are removed from state-store.
   @Test(timeout = 15000)
   public void testRemoveExpiredMasterKeyInRMStateStore() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
     memStore.init(testConf);
     RMState rmState = memStore.getState();
 


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/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 40e1e94..82ddb54 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
@@ -45,10 +45,10 @@ import org.apache.hadoop.yarn.server.webapp.AppAttemptBlock;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -86,7 +86,7 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         .th(".resource", "ResourceName").th(".capacity", "Capability")
         .th(".containers", "NumContainers")
         .th(".relaxlocality", "RelaxLocality")
-        .th(".labelexpression", "NodeLabelExpression")._()._().tbody();
+        .th(".labelexpression", "NodeLabelExpression").__().__().tbody();
 
     StringBuilder resourceRequestTableData = new StringBuilder("[\n");
     for (ResourceRequestInfo resourceRequest  : resourceRequests) {
@@ -114,9 +114,9 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
     }
     resourceRequestTableData.append("]");
     html.script().$type("text/javascript")
-        ._("var resourceRequestsTableData=" + resourceRequestTableData)._();
-    tbody._()._();
-    div._();
+        .__("var resourceRequestsTableData=" + resourceRequestTableData).__();
+    tbody.__().__();
+    div.__();
   }
 
   private Resource getTotalResource(List<ResourceRequestInfo> requests) {
@@ -163,7 +163,7 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
         th(_TH, "Node Local Request").
         th(_TH, "Rack Local Request").
         th(_TH, "Off Switch Request").
-      _();
+        __();
 
     String[] containersType =
         { "Num Node Local Containers (satisfied by)", "Num Rack Local Containers (satisfied by)",
@@ -173,10 +173,10 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       table.tr((odd = !odd) ? _ODD : _EVEN).td(containersType[i])
         .td(String.valueOf(attemptMetrics.getLocalityStatistics()[i][0]))
         .td(i == 0 ? "" : String.valueOf(attemptMetrics.getLocalityStatistics()[i][1]))
-        .td(i <= 1 ? "" : String.valueOf(attemptMetrics.getLocalityStatistics()[i][2]))._();
+        .td(i <= 1 ? "" : String.valueOf(attemptMetrics.getLocalityStatistics()[i][2])).__();
     }
-    table._();
-    div._();
+    table.__();
+    div.__();
   }
 
   private boolean isApplicationInFinalState(YarnApplicationAttemptState state) {
@@ -192,12 +192,12 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
       if (!isApplicationInFinalState(YarnApplicationAttemptState
           .valueOf(attempt.getAppAttemptState().toString()))) {
         RMAppAttemptMetrics metrics = attempt.getRMAppAttemptMetrics();
-        DIV<Hamlet> pdiv = html._(InfoBlock.class).div(_INFO_WRAP);
+        DIV<Hamlet> pdiv = html.__(InfoBlock.class).div(_INFO_WRAP);
         info("Application Attempt Overview").clear();
-        info("Application Attempt Metrics")._(
+        info("Application Attempt Metrics").__(
           "Application Attempt Headroom : ", metrics == null ? "N/A" :
             metrics.getApplicationAttemptHeadroom());
-        pdiv._();
+        pdiv.__();
       }
     }
   }
@@ -226,23 +226,23 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
           .getBlacklistUpdates().getBlacklistAdditions());
 
     info("Application Attempt Overview")
-      ._(
+      .__(
         "Application Attempt State:",
         appAttempt.getAppAttemptState() == null ? UNAVAILABLE : appAttempt
           .getAppAttemptState())
-        ._("Started:", Times.format(appAttempt.getStartedTime()))
-        ._("Elapsed:",
+        .__("Started:", Times.format(appAttempt.getStartedTime()))
+        .__("Elapsed:",
             org.apache.hadoop.util.StringUtils.formatTime(Times.elapsed(
                 appAttempt.getStartedTime(), appAttempt.getFinishedTime())))
-      ._(
+      .__(
         "AM Container:",
         appAttempt.getAmContainerId() == null || containers == null
             || !hasAMContainer(appAttemptReport.getAMContainerId(), containers)
             ? null : root_url("container", appAttempt.getAmContainerId()),
         appAttempt.getAmContainerId() == null ? "N/A" :
           String.valueOf(appAttempt.getAmContainerId()))
-      ._("Node:", node)
-      ._(
+      .__("Node:", node)
+      .__(
         "Tracking URL:",
         appAttempt.getTrackingUrl() == null
             || appAttempt.getTrackingUrl().equals(UNAVAILABLE) ? null
@@ -254,12 +254,12 @@ public class RMAppAttemptBlock extends AppAttemptBlock{
                 || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.FAILED
                 || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.KILLED
                 ? "History" : "ApplicationMaster")
-      ._(
+      .__(
         "Diagnostics Info:",
         appAttempt.getDiagnosticsInfo() == null ? "" : appAttempt
           .getDiagnosticsInfo())
-      ._("Nodes blacklisted by the application:", appBlacklistedNodes)
-      ._("Nodes blacklisted by the system:", rmBlackListedNodes);
+      .__("Nodes blacklisted by the application:", appBlacklistedNodes)
+      .__("Nodes blacklisted by the system:", rmBlackListedNodes);
   }
 
   private String getNodeString(Collection<String> nodes) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.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/RMAppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
index e5d6c16..cd04264 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppBlock.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptM
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.AppBlock;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -82,33 +82,33 @@ public class RMAppBlock extends AppBlock{
         attemptMetrics == null ? 0 : attemptMetrics
           .getNumNonAMContainersPreempted();
     DIV<Hamlet> pdiv = html.
-        _(InfoBlock.class).
+        __(InfoBlock.class).
         div(_INFO_WRAP);
     info("Application Overview").clear();
     info("Application Metrics")
-        ._("Total Resource Preempted:",
+        .__("Total Resource Preempted:",
           appMetrics == null ? "N/A" : appMetrics.getResourcePreempted())
-        ._("Total Number of Non-AM Containers Preempted:",
+        .__("Total Number of Non-AM Containers Preempted:",
           appMetrics == null ? "N/A"
               : appMetrics.getNumNonAMContainersPreempted())
-        ._("Total Number of AM Containers Preempted:",
+        .__("Total Number of AM Containers Preempted:",
           appMetrics == null ? "N/A"
               : appMetrics.getNumAMContainersPreempted())
-        ._("Resource Preempted from Current Attempt:",
+        .__("Resource Preempted from Current Attempt:",
           attemptResourcePreempted)
-        ._("Number of Non-AM Containers Preempted from Current Attempt:",
+        .__("Number of Non-AM Containers Preempted from Current Attempt:",
           attemptNumNonAMContainerPreempted)
-        ._("Aggregate Resource Allocation:",
+        .__("Aggregate Resource Allocation:",
           String.format("%d MB-seconds, %d vcore-seconds",
               appMetrics == null ? "N/A" : appMetrics.getMemorySeconds(),
               appMetrics == null ? "N/A" : appMetrics.getVcoreSeconds()))
-        ._("Aggregate Preempted Resource Allocation:",
+        .__("Aggregate Preempted Resource Allocation:",
           String.format("%d MB-seconds, %d vcore-seconds",
             appMetrics == null ? "N/A" : appMetrics.getPreemptedMemorySeconds(),
             appMetrics == null ? "N/A" :
                 appMetrics.getPreemptedVcoreSeconds()));
 
-    pdiv._();
+    pdiv.__();
   }
 
   @Override
@@ -122,7 +122,7 @@ public class RMAppBlock extends AppBlock{
             .th(".appBlacklistednodes", "Nodes blacklisted by the application",
                 "Nodes blacklisted by the app")
             .th(".rmBlacklistednodes", "Nodes blacklisted by the RM for the"
-                + " app", "Nodes blacklisted by the system")._()._().tbody();
+                + " app", "Nodes blacklisted by the system").__().__().tbody();
 
     RMApp rmApp = this.rm.getRMContext().getRMApps().get(this.appID);
     if (rmApp == null) {
@@ -174,9 +174,9 @@ public class RMAppBlock extends AppBlock{
     }
     attemptsTableData.append("]");
     html.script().$type("text/javascript")
-        ._("var attemptsTableData=" + attemptsTableData)._();
+        .__("var attemptsTableData=" + attemptsTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.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/RMAppLogAggregationStatusBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
index f7f7c97..c1f2e5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppLogAggregationStatusBlock.java
@@ -38,9 +38,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.util.Apps;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -87,28 +87,28 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
       tr().
         th(_TH, "Log Aggregation Status").
         th(_TH, "Description").
-      _();
+        __();
     table_description.tr().td(LogAggregationStatus.DISABLED.name())
-      .td("Log Aggregation is Disabled.")._();
+      .td("Log Aggregation is Disabled.").__();
     table_description.tr().td(LogAggregationStatus.NOT_START.name())
-      .td("Log Aggregation does not Start.")._();
+      .td("Log Aggregation does not Start.").__();
     table_description.tr().td(LogAggregationStatus.RUNNING.name())
-      .td("Log Aggregation is Running.")._();
+      .td("Log Aggregation is Running.").__();
     table_description.tr().td(LogAggregationStatus.RUNNING_WITH_FAILURE.name())
       .td("Log Aggregation is Running, but has failures "
-          + "in previous cycles")._();
+          + "in previous cycles").__();
     table_description.tr().td(LogAggregationStatus.SUCCEEDED.name())
       .td("Log Aggregation is Succeeded. All of the logs have been "
-          + "aggregated successfully.")._();
+          + "aggregated successfully.").__();
     table_description.tr().td(LogAggregationStatus.FAILED.name())
       .td("Log Aggregation is Failed. At least one of the logs "
-          + "have not been aggregated.")._();
+          + "have not been aggregated.").__();
     table_description.tr().td(LogAggregationStatus.TIME_OUT.name())
       .td("The application is finished, but the log aggregation status is "
           + "not updated for a long time. Not sure whether the log aggregation "
-          + "is finished or not.")._();
-    table_description._();
-    div_description._();
+          + "is finished or not.").__();
+    table_description.__();
+    div_description.__();
 
     RMApp rmApp = rm.getRMContext().getRMApps().get(appId);
     // Application Log aggregation status Table
@@ -131,7 +131,7 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
       .th(_TH, "Last "
           + maxLogAggregationDiagnosticsInMemory + " Diagnostic Messages")
       .th(_TH, "Last "
-          + maxLogAggregationDiagnosticsInMemory + " Failure Messages")._();
+          + maxLogAggregationDiagnosticsInMemory + " Failure Messages").__();
 
     if (rmApp != null) {
       Map<NodeId, LogAggregationReport> logAggregationReports =
@@ -152,11 +152,11 @@ public class RMAppLogAggregationStatusBlock extends HtmlBlock {
             .td(report.getKey().toString())
             .td(status == null ? "N/A" : status.toString())
             .td(message == null ? "N/A" : message)
-            .td(failureMessage == null ? "N/A" : failureMessage)._();
+            .td(failureMessage == null ? "N/A" : failureMessage).__();
         }
       }
     }
-    table._();
-    div._();
+    table.__();
+    div.__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.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/RMAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
index 61674d2..ede71e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMAppsBlock.java
@@ -35,11 +35,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.View;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 
 import com.google.inject.Inject;
 
@@ -72,8 +71,8 @@ public class RMAppsBlock extends AppsBlock {
           .th(".clusterPercentage", "% of Cluster")
           .th(".progress", "Progress")
           .th(".ui", "Tracking UI")
-          .th(".blacklisted", "Blacklisted Nodes")._()
-          ._().tbody();
+          .th(".blacklisted", "Blacklisted Nodes").__()
+          .__().tbody();
 
     StringBuilder appsTableData = new StringBuilder("[\n");
     for (ApplicationReport appReport : appReports) {
@@ -190,8 +189,8 @@ public class RMAppsBlock extends AppsBlock {
     }
     appsTableData.append("]");
     html.script().$type("text/javascript")
-      ._("var appsTableData=" + appsTableData)._();
+      .__("var appsTableData=" + appsTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMErrorsAndWarningsPage.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/RMErrorsAndWarningsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMErrorsAndWarningsPage.java
index 216deeb..c2ac59d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMErrorsAndWarningsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMErrorsAndWarningsPage.java
@@ -31,7 +31,7 @@ public class RMErrorsAndWarningsPage extends RmView {
   }
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String title = "Errors and Warnings in the ResourceManager";
     setTitle(title);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RedirectionErrorPage.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/RedirectionErrorPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RedirectionErrorPage.java
index beb0cca..d81e64d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RedirectionErrorPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RedirectionErrorPage.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
  * because of a redirection issue.
  */
 public class RedirectionErrorPage extends RmView {
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String aid = $(YarnWebParams.APPLICATION_ID);
 
     commonPreHead(html);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.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/RmView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
index 1a437f8..fc844f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
@@ -33,7 +33,7 @@ public class RmView extends TwoColumnLayout {
   static final int MAX_FAST_ROWS = 1000;    // inline js array
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "apps");
     set(initID(DATATABLES, "apps"), initAppsTable());
@@ -45,7 +45,7 @@ public class RmView extends TwoColumnLayout {
     setTitle(sjoin(reqState, "Applications"));
   }
 
-  protected void commonPreHead(Page.HTML<_> html) {
+  protected void commonPreHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.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/SchedulerPageUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.java
index 99c0565..8c2d271 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/SchedulerPageUtil.java
@@ -27,7 +27,7 @@ public class SchedulerPageUtil {
     private void reopenQueue(Block html) {
       html.
           script().$type("text/javascript").
-          _("function reopenQueryNodes() {",
+          __("function reopenQueryNodes() {",
             "  var currentParam = window.location.href.split('?');",
             "  var tmpCurrentParam = currentParam;",
             "  var queryQueuesString = '';",
@@ -52,13 +52,13 @@ public class SchedulerPageUtil {
             "                  'open_node.jstree' :function(e, data) { storeExpandedQueue(e, data); },",
             "                  'close_node.jstree':function(e, data) { storeExpandedQueue(e, data); }",
             "  });",
-            "}")._();
+            "}").__();
     }
 
     private void storeExpandedQueue (Block html) {
       html.
           script().$type("text/javascript").
-          _("function storeExpandedQueue(e, data) {",
+          __("function storeExpandedQueue(e, data) {",
             "  var OPEN_QUEUES = 'openQueues';",
             "  var ACTION_OPEN = 'open';",
             "  var ACTION_CLOSED = 'closed';",
@@ -166,7 +166,7 @@ public class SchedulerPageUtil {
             "    queryString = queryString + '#' + queueName;",
             "  }",
             "  return queryString;",
-            "}")._();
+            "}").__();
     }
 
     @Override protected void render(Block html) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
index 27944d3..cec085b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-sharedcachemanager/src/main/java/org/apache/hadoop/yarn/server/sharedcachemanager/webapp/SCMOverviewPage.java
@@ -43,7 +43,7 @@ import com.google.inject.Inject;
 @Unstable
 public class SCMOverviewPage extends TwoColumnLayout {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }
@@ -60,9 +60,9 @@ public class SCMOverviewPage extends TwoColumnLayout {
   static private class SCMOverviewNavBlock extends HtmlBlock {
     @Override
     protected void render(Block html) {
-      html.div("#nav").h3("Tools").ul().li().a("/conf", "Configuration")._()
-          .li().a("/stacks", "Thread dump")._().li().a("/logs", "Logs")._()
-          .li().a("/metrics", "Metrics")._()._()._();
+      html.div("#nav").h3("Tools").ul().li().a("/conf", "Configuration").__()
+          .li().a("/stacks", "Thread dump").__().li().a("/logs", "Logs").__()
+          .li().a("/metrics", "Metrics").__().__().__();
     }
   }
 
@@ -81,15 +81,15 @@ public class SCMOverviewPage extends TwoColumnLayout {
           CleanerMetrics.getInstance(), ClientSCMMetrics.getInstance(),
               SharedCacheUploaderMetrics.getInstance());
       info("Shared Cache Manager overview").
-          _("Started on:", Times.format(scm.getStartTime())).
-          _("Cache hits: ", metricsInfo.getCacheHits()).
-          _("Cache misses: ", metricsInfo.getCacheMisses()).
-          _("Cache releases: ", metricsInfo.getCacheReleases()).
-          _("Accepted uploads: ", metricsInfo.getAcceptedUploads()).
-          _("Rejected uploads: ", metricsInfo.getRejectUploads()).
-          _("Deleted files by the cleaner: ", metricsInfo.getTotalDeletedFiles()).
-          _("Processed files by the cleaner: ", metricsInfo.getTotalProcessedFiles());
-      html._(InfoBlock.class);
+          __("Started on:", Times.format(scm.getStartTime())).
+          __("Cache hits: ", metricsInfo.getCacheHits()).
+          __("Cache misses: ", metricsInfo.getCacheMisses()).
+          __("Cache releases: ", metricsInfo.getCacheReleases()).
+          __("Accepted uploads: ", metricsInfo.getAcceptedUploads()).
+          __("Rejected uploads: ", metricsInfo.getRejectUploads()).
+          __("Deleted files by the cleaner: ", metricsInfo.getTotalDeletedFiles()).
+          __("Processed files by the cleaner: ", metricsInfo.getTotalProcessedFiles());
+      html.__(InfoBlock.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java
index 7d61f74..4886c55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/ProxyUtils.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.yarn.server.webproxy;
 
 import org.apache.hadoop.yarn.webapp.MimeType;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,7 +41,7 @@ public class ProxyUtils {
       "This filter only works for HTTP/HTTPS";
   public static final String LOCATION = "Location";
 
-  public static class _ implements Hamlet._ {
+  public static class __ implements Hamlet.__ {
     //Empty
   }
 
@@ -50,7 +50,7 @@ public class ProxyUtils {
       super(out, 0, false);
     }
 
-    public HTML<ProxyUtils._> html() {
+    public HTML<ProxyUtils.__> html() {
       return new HTML<>("html", null, EnumSet.of(EOpt.ENDTAG));
     }
   }
@@ -86,13 +86,13 @@ public class ProxyUtils {
     PrintWriter writer = response.getWriter();
     Page p = new Page(writer);
     p.html()
-        .head().title("Moved")._()
+        .head().title("Moved").__()
         .body()
         .h1("Moved")
         .div()
-          ._("Content has moved ")
-          .a(location, "here")._()
-        ._()._();
+          .__("Content has moved ")
+          .a(location, "here").__()
+        .__().__();
     writer.close();
   }
 
@@ -110,7 +110,7 @@ public class ProxyUtils {
     Page p = new Page(resp.getWriter());
     p.html().
         h1(message).
-         _();
+        __();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index b32ee30..e1588c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.util.TrackingUriPlugin;
 import org.apache.hadoop.yarn.webapp.MimeType;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.http.Header;
 import org.apache.http.HttpResponse;
@@ -108,7 +108,7 @@ public class WebAppProxyServlet extends HttpServlet {
   /**
    * Empty Hamlet class.
    */
-  private static class _ implements Hamlet._ {
+  private static class __ implements Hamlet.__ {
     //Empty
   }
   
@@ -117,7 +117,7 @@ public class WebAppProxyServlet extends HttpServlet {
       super(out, 0, false);
     }
   
-    public HTML<WebAppProxyServlet._> html() {
+    public HTML<WebAppProxyServlet.__> html() {
       return new HTML<>("html", null, EnumSet.of(EOpt.ENDTAG));
     }
   }
@@ -172,10 +172,10 @@ public class WebAppProxyServlet extends HttpServlet {
     p.html().
       h1("WARNING: The following page may not be safe!").
       h3().
-      _("click ").a(link, "here").
-      _(" to continue to an Application Master web interface owned by ", user).
-      _().
-    _();
+        __("click ").a(link, "here").
+        __(" to continue to an Application Master web interface owned by ", user).
+        __().
+        __();
   }
   
   /**


---------------------------------------------------------------------
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: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 24792bb..4bae71e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -17,16 +17,109 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_KEY;
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
+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.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.ContentSummary;
+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;
+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.AddingECPolicyResponse;
+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.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+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.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+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.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.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
 
 /**
  * This class is responsible for handling all of the RPC calls to the It is
@@ -36,10 +129,42 @@ import com.google.common.annotations.VisibleForTesting;
  * the requests to the active
  * {@link org.apache.hadoop.hdfs.server.namenode.NameNode NameNode}.
  */
-public class RouterRpcServer extends AbstractService {
+public class RouterRpcServer extends AbstractService implements ClientProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterRpcServer.class);
+
+
+  /** 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;
 
   /** The RPC server that listens to requests from clients. */
   private final Server rpcServer;
+  /** The address for this RPC server. */
+  private final InetSocketAddress rpcAddress;
+
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+
+
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+  /** Interface to map global name space to HDFS subcluster name spaces. */
+  private final FileSubclusterResolver subclusterResolver;
+
+
+  /** Category of the operation that a thread is executing. */
+  private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
+
 
   /**
    * Construct a router RPC server.
@@ -54,31 +179,94 @@ public class RouterRpcServer extends AbstractService {
           throws IOException {
     super(RouterRpcServer.class.getName());
 
-    this.rpcServer = null;
-  }
+    this.conf = configuration;
+    this.router = router;
+    this.namenodeResolver = nnResolver;
+    this.subclusterResolver = fileResolver;
 
-  /**
-   * Allow access to the client RPC server for testing.
-   *
-   * @return The RPC server.
-   */
-  @VisibleForTesting
-  public Server getServer() {
-    return this.rpcServer;
+    // 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);
+
+    int readerCount = this.conf.getInt(DFS_ROUTER_READER_COUNT_KEY,
+        DFS_ROUTER_READER_COUNT_DEFAULT);
+
+    int handlerQueueSize = this.conf.getInt(DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY,
+        DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT);
+
+    // Override Hadoop Common IPC setting
+    int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY,
+        DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY,
+        readerQueueSize);
+
+    RPC.setProtocolEngine(this.conf, ClientNamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+
+    ClientNamenodeProtocolServerSideTranslatorPB
+        clientProtocolServerTranslator =
+            new ClientNamenodeProtocolServerSideTranslatorPB(this);
+    BlockingService clientNNPbService = ClientNamenodeProtocol
+        .newReflectiveBlockingService(clientProtocolServerTranslator);
+
+    InetSocketAddress confRpcAddress = conf.getSocketAddr(
+        DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY,
+        DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY,
+        DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_DEFAULT,
+        DFSConfigKeys.DFS_ROUTER_RPC_PORT_DEFAULT);
+    LOG.info("RPC server binding to {} with {} handlers for Router {}",
+        confRpcAddress, handlerCount, this.router.getRouterId());
+
+    this.rpcServer = new RPC.Builder(this.conf)
+        .setProtocol(ClientNamenodeProtocolPB.class)
+        .setInstance(clientNNPbService)
+        .setBindAddress(confRpcAddress.getHostName())
+        .setPort(confRpcAddress.getPort())
+        .setNumHandlers(handlerCount)
+        .setnumReaders(readerCount)
+        .setQueueSizePerHandler(handlerQueueSize)
+        .setVerbose(false)
+        .build();
+    // We don't want the server to log the full stack trace for some exceptions
+    this.rpcServer.addTerseExceptions(
+        RemoteException.class,
+        StandbyException.class,
+        SafeModeException.class,
+        FileNotFoundException.class,
+        FileAlreadyExistsException.class,
+        AccessControlException.class,
+        LeaseExpiredException.class,
+        NotReplicatedYetException.class,
+        IOException.class);
+
+    // The RPC-server port can be ephemeral... ensure we have the correct info
+    InetSocketAddress listenAddress = this.rpcServer.getListenerAddress();
+    this.rpcAddress = new InetSocketAddress(
+        confRpcAddress.getHostName(), listenAddress.getPort());
+
+    // Create the client
+    this.rpcClient = new RouterRpcClient(this.conf, this.router.getRouterId(),
+        this.namenodeResolver);
   }
 
   @Override
   protected void serviceInit(Configuration configuration) throws Exception {
+    this.conf = configuration;
     super.serviceInit(configuration);
   }
 
-  /**
-   * Start client and service RPC servers.
-   */
   @Override
   protected void serviceStart() throws Exception {
     if (this.rpcServer != null) {
       this.rpcServer.start();
+      LOG.info("Router RPC up at: {}", this.getRpcAddress());
     }
     super.serviceStart();
   }
@@ -92,11 +280,1652 @@ public class RouterRpcServer extends AbstractService {
   }
 
   /**
-   * Wait until the RPC servers have shutdown.
+   * Get the RPC client to the Namenode.
+   *
+   * @return RPC clients to the Namenodes.
    */
-  void join() throws InterruptedException {
-    if (this.rpcServer != null) {
-      this.rpcServer.join();
+  public RouterRpcClient getRPCClient() {
+    return rpcClient;
+  }
+
+  /**
+   * Allow access to the client RPC server for testing.
+   *
+   * @return The RPC server.
+   */
+  @VisibleForTesting
+  public Server getServer() {
+    return rpcServer;
+  }
+
+  /**
+   * Get the RPC address of the service.
+   *
+   * @return RPC service address.
+   */
+  public InetSocketAddress getRpcAddress() {
+    return rpcAddress;
+  }
+
+  /**
+   * Check if the Router is in safe mode. We should only see READ, WRITE, and
+   * UNCHECKED. It includes a default handler when we haven't implemented an
+   * operation. If not supported, it always throws an exception reporting the
+   * operation.
+   *
+   * @param op Category of the operation to check.
+   * @param supported If the operation is supported or not. If not, it will
+   *                  throw an UnsupportedOperationException.
+   * @throws StandbyException If the Router is in safe mode and cannot serve
+   *                          client requests.
+   * @throws UnsupportedOperationException If the operation is not supported.
+   */
+  private void checkOperation(OperationCategory op, boolean supported)
+      throws StandbyException, UnsupportedOperationException {
+    checkOperation(op);
+
+    if (!supported) {
+      String methodName = getMethodName();
+      throw new UnsupportedOperationException(
+          "Operation \"" + methodName + "\" is not supported");
+    }
+  }
+
+  /**
+   * Check if the Router is in safe mode. We should only see READ, WRITE, and
+   * UNCHECKED. This function should be called by all ClientProtocol functions.
+   *
+   * @param op Category of the operation to check.
+   * @throws StandbyException If the Router is in safe mode and cannot serve
+   *                          client requests.
+   */
+  private void checkOperation(OperationCategory op) throws StandbyException {
+    // Log the function we are currently calling.
+    if (LOG.isDebugEnabled()) {
+      String methodName = getMethodName();
+      LOG.debug("Proxying operation: {}", methodName);
+    }
+
+    // Store the category of the operation category for this thread
+    opCategory.set(op);
+
+    // We allow unchecked and read operations
+    if (op == OperationCategory.UNCHECKED || op == OperationCategory.READ) {
+      return;
+    }
+
+    // TODO check Router safe mode and return Standby exception
+  }
+
+  @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;
+  }
+
+  @Override // ClientProtocol
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override // ClientProtocol
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @Override // ClientProtocol
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize,
+      CryptoProtocolVersion[] supportedVersions, String ecPolicyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    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 location to create a file. It checks if the file already existed
+   * in one of the locations.
+   *
+   * @param src Path of the file to check.
+   * @return The remote location for this file.
+   * @throws IOException If the file has no creation location.
+   */
+  private RemoteLocation getCreateLocation(final String src)
+      throws IOException {
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    if (locations == null || locations.isEmpty()) {
+      throw new IOException("Cannot get locations to create " + src);
+    }
+
+    RemoteLocation createLocation = locations.get(0);
+    if (locations.size() > 1) {
+      try {
+        // Check if this file already exists in other subclusters
+        LocatedBlocks existingLocation = getBlockLocations(src, 0, 1);
+        if (existingLocation != null) {
+          // Forward to the existing location and let the NN handle the error
+          LocatedBlock existingLocationLastLocatedBlock =
+              existingLocation.getLastLocatedBlock();
+          if (existingLocationLastLocatedBlock == null) {
+            // The block has no blocks yet, check for the meta data
+            for (RemoteLocation location : locations) {
+              RemoteMethod method = new RemoteMethod("getFileInfo",
+                  new Class<?>[] {String.class}, new RemoteParam());
+              if (rpcClient.invokeSingle(location, method) != null) {
+                createLocation = location;
+                break;
+              }
+            }
+          } else {
+            ExtendedBlock existingLocationLastBlock =
+                existingLocationLastLocatedBlock.getBlock();
+            String blockPoolId = existingLocationLastBlock.getBlockPoolId();
+            createLocation = getLocationForPath(src, true, blockPoolId);
+          }
+        }
+      } catch (FileNotFoundException fne) {
+        // Ignore if the file is not found
+      }
+    }
+    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 (LastBlockWithStatus) rpcClient.invokeSequential(
+        locations, method, LastBlockWithStatus.class, null);
+  }
+
+  // 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;
+  }
+
+  @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;
+  }
+
+  @Override
+  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);
+  }
+
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    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 {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("setPermission",
+        new Class<?>[] {String.class, FsPermission.class},
+        new RemoteParam(), permissions);
+    rpcClient.invokeSequential(locations, method);
+  }
+
+  @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);
+    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 // ClientProtocol
+  public LocatedBlock addBlock(String src, String clientName,
+      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);
+  }
+
+  /**
+   * Excluded nodes are not verified and will be ignored by placement if they
+   * are not in the same nameservice as the file.
+   */
+  @Override // ClientProtocol
+  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 {
+    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 {
+    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 {
+    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 {
+    checkOperation(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 // ClientProtocol
+  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);
+  }
+
+  @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;
+  }
+
+  @Deprecated
+  @Override // ClientProtocol
+  public boolean rename(final String src, final String dst)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations = getLocationsForPath(src, true);
+    // 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 {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> srcLocations = getLocationsForPath(src, true);
+    // 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 {
+    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 {
+    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 {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    RemoteMethod method = new RemoteMethod("delete",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        recursive);
+    return ((Boolean) rpcClient.invokeSequential(locations, method,
+        Boolean.class, Boolean.TRUE)).booleanValue();
+  }
+
+  @Override // ClientProtocol
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, true);
+    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);
+    RemoteMethod method = new RemoteMethod("mkdirs",
+        new Class<?>[] {String.class, FsPermission.class, boolean.class},
+        new RemoteParam(), masked, createParent);
+    return ((Boolean) rpcClient.invokeSingle(firstLocation, method))
+        .booleanValue();
+  }
+
+  @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);
+  }
+
+  @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, Object> listings =
+        rpcClient.invokeConcurrent(locations, method, false, false);
+
+    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, Object> entry : listings.entrySet()) {
+        RemoteLocation location = entry.getKey();
+        DirectoryListing listing = (DirectoryListing) 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 {
+    checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getFileInfo",
+        new Class<?>[] {String.class}, new RemoteParam());
+    HdfsFileStatus 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;
+  }
+
+  @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();
+  }
+
+  @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);
+  }
+
+  @Override // ClientProtocol
+  public long[] getStats() throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    RemoteMethod method = new RemoteMethod("getStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    long[] combinedData = new long[STATS_ARRAY_LENGTH];
+    for (Object o : results.values()) {
+      long[] data = (long[]) o;
+      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 {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeInfo> datanodesMap = new LinkedHashMap<>();
+    RemoteMethod method = new RemoteMethod("getDatanodeReport",
+        new Class<?>[] {DatanodeReportType.class}, type);
+
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    for (Entry<FederationNamespaceInfo, Object> entry : results.entrySet()) {
+      FederationNamespaceInfo ns = entry.getKey();
+      DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue();
+      for (DatanodeInfo node : result) {
+        String nodeId = node.getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          // Add the subcluster as a suffix to the network location
+          node.setNetworkLocation(
+              NodeBase.PATH_SEPARATOR_STR + ns.getNameserviceId() +
+              node.getNetworkLocation());
+          datanodesMap.put(nodeId, node);
+        } else {
+          LOG.debug("{} is in multiple subclusters", nodeId);
+        }
+      }
+    }
+    // Map -> Array
+    Collection<DatanodeInfo> datanodes = datanodesMap.values();
+    DatanodeInfo[] combinedData = new DatanodeInfo[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
+  }
+
+  @Override // ClientProtocol
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) throws IOException {
+    checkOperation(OperationCategory.UNCHECKED);
+
+    Map<String, DatanodeStorageReport> datanodesMap = new HashMap<>();
+    RemoteMethod method = new RemoteMethod("getDatanodeStorageReport",
+        new Class<?>[] {DatanodeReportType.class}, type);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+    for (Object r : results.values()) {
+      DatanodeStorageReport[] result = (DatanodeStorageReport[]) r;
+      for (DatanodeStorageReport node : result) {
+        String nodeId = node.getDatanodeInfo().getXferAddr();
+        if (!datanodesMap.containsKey(nodeId)) {
+          datanodesMap.put(nodeId, node);
+        }
+        // TODO merge somehow, right now it just takes the first one
+      }
+    }
+
+    Collection<DatanodeStorageReport> datanodes = datanodesMap.values();
+    // TODO sort somehow
+    DatanodeStorageReport[] combinedData =
+        new DatanodeStorageReport[datanodes.size()];
+    combinedData = datanodes.toArray(combinedData);
+    return combinedData;
+  }
+
+  @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, Object> results =
+        rpcClient.invokeConcurrent(nss, method, true, true);
+
+    // We only report true if all the name space are in safe mode
+    int numSafemode = 0;
+    for (Object result : results.values()) {
+      if (result instanceof Boolean) {
+        boolean safemode = (boolean) result;
+        if (safemode) {
+          numSafemode++;
+        }
+      }
     }
+    return numSafemode == results.size();
+  }
+
+  @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, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    boolean success = true;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Boolean> results =
+        (Map<FederationNamespaceInfo, Boolean>)obj;
+    Collection<Boolean> sucesses = results.values();
+    for (boolean s : sucesses) {
+      if (!s) {
+        success = false;
+      }
+    }
+    return success;
+  }
+
+  @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, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    boolean success = true;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Boolean> results =
+        (Map<FederationNamespaceInfo, Boolean>)obj;
+    Collection<Boolean> sucesses = results.values();
+    for (boolean s : sucesses) {
+      if (!s) {
+        success = false;
+      }
+    }
+    return success;
+  }
+
+  @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, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the maximum txid
+    long txid = 0;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Long> results =
+        (Map<FederationNamespaceInfo, Long>)obj;
+    Collection<Long> txids = results.values();
+    for (long t : txids) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the first rolling upgrade info
+    RollingUpgradeInfo info = null;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> results =
+        (Map<FederationNamespaceInfo, RollingUpgradeInfo>)obj;
+    Collection<RollingUpgradeInfo> infos = results.values();
+    for (RollingUpgradeInfo infoNs : infos) {
+      if (info == null && infoNs != null) {
+        info = infoNs;
+      }
+    }
+    return info;
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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());
+      @SuppressWarnings("unchecked")
+      Map<String, ContentSummary> results =
+          (Map<String, ContentSummary>) ((Object)rpcClient.invokeConcurrent(
+              locations, method, false, false));
+      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 results 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 {
+    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 {
+    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 {
+    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 {
+    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 {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // Client Protocol
+  public void disallowSnapshot(String snapshot) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // Client Protocol
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String earlierSnapshotName, String laterSnapshotName) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public long addCacheDirective(CacheDirectiveInfo path,
+      EnumSet<CacheFlag> flags) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return 0;
+  }
+
+  @Override // ClientProtocol
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void removeCacheDirective(long id) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void removeCachePool(String cachePoolName) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<EncryptionZone> listEncryptionZones(long prevId)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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, Object> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false);
+
+    // Return the maximum txid
+    long txid = 0;
+    Object obj = ret;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Long> results =
+        (Map<FederationNamespaceInfo, Long>)obj;
+    Collection<Long> txids = results.values();
+    for (long t : txids) {
+      if (t > txid) {
+        txid = t;
+      }
+    }
+    return txid;
+  }
+
+  @Override // ClientProtocol
+  public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE);
+    return null;
+  }
+
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public AddingECPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+    return null;
+  }
+
+  @Override // ClientProtocol
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override // ClientProtocol
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws IOException {
+    checkOperation(OperationCategory.WRITE);
+
+    // TODO assign global replicas instead of applying them to each folder
+    final List<RemoteLocation> locations = getLocationsForPath(path, true);
+    RemoteMethod method = new RemoteMethod("setQuota",
+        new Class<?>[] {String.class, Long.class, Long.class,
+            StorageType.class},
+        new RemoteParam(), namespaceQuota, storagespaceQuota, type);
+    rpcClient.invokeConcurrent(locations, method, false, false);
+  }
+
+  @Override // ClientProtocol
+  public QuotaUsage getQuotaUsage(String path) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  @Override
+  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);
+    }
+  }
+
+  @Override
+  public void unsetStoragePolicy(String src) throws IOException {
+    checkOperation(OperationCategory.WRITE, false);
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    checkOperation(OperationCategory.READ, false);
+    return null;
+  }
+
+  /**
+   * Locate the location with the matching block pool id.
+   *
+   * @param path Path to check.
+   * @param failIfLocked Fail the request if locked (top mount point).
+   * @param blockPoolId The block pool ID of the namespace to search for.
+   * @return Prioritized list of locations in the federated cluster.
+   * @throws IOException if the location for this path cannot be determined.
+   */
+  private RemoteLocation getLocationForPath(
+      String path, boolean failIfLocked, String blockPoolId)
+          throws IOException {
+
+    final List<RemoteLocation> locations =
+        getLocationsForPath(path, failIfLocked);
+
+    String nameserviceId = null;
+    Set<FederationNamespaceInfo> namespaces =
+        this.namenodeResolver.getNamespaces();
+    for (FederationNamespaceInfo namespace : namespaces) {
+      if (namespace.getBlockPoolId().equals(blockPoolId)) {
+        nameserviceId = namespace.getNameserviceId();
+        break;
+      }
+    }
+    if (nameserviceId != null) {
+      for (RemoteLocation location : locations) {
+        if (location.getNameserviceId().equals(nameserviceId)) {
+          return location;
+        }
+      }
+    }
+    throw new IOException(
+        "Cannot locate a nameservice for block pool " + blockPoolId);
+  }
+
+  /**
+   * Get the possible locations of a path in the federated cluster.
+   *
+   * @param path Path to check.
+   * @param failIfLocked Fail the request if locked (top mount point).
+   * @return Prioritized list of locations in the federated cluster.
+   * @throws IOException If the location for this path cannot be determined.
+   */
+  private List<RemoteLocation> getLocationsForPath(
+      String path, boolean failIfLocked) throws IOException {
+    // Check the location for this path
+    final PathLocation location =
+        this.subclusterResolver.getDestinationForPath(path);
+    if (location == null) {
+      throw new IOException("Cannot find locations for " + path + " in " +
+          this.subclusterResolver);
+    }
+
+    // Log the access to a path
+    return location.getDestinations();
+  }
+
+  /**
+   * 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<>();
+    // TODO add when we have a Mount Table
+    return ret;
+  }
+
+  /**
+   * Create a new file status for a mount point.
+   *
+   * @param name Name of the mount point.
+   * @param childrenNum Number of children.
+   * @param dates 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(0, true, 0, 0, modTime, accessTime, permission,
+        owner, group, new byte[0], DFSUtil.string2Bytes(name), inodeId,
+        childrenNum, null, (byte) 0, null);
+  }
+
+  /**
+   * 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.
+   * @throws IOException If we cannot get the user information.
+   */
+  static UserGroupInformation getRemoteUser() throws IOException {
+    UserGroupInformation ugi = Server.getRemoteUser();
+    return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index cfb029b..10ed7f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4544,6 +4544,101 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.default.nameserviceId</name>
+    <value></value>
+    <description>
+      Nameservice identifier of the default subcluster to monitor.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc.enable</name>
+    <value>true</value>
+    <description>
+      If the RPC service to handle client requests in the router is enabled.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc-address</name>
+    <value>0.0.0.0:8888</value>
+    <description>
+      RPC address that handles all clients requests.
+      The value of this property will take the form of router-host1:rpc-port.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.rpc-bind-host</name>
+    <value></value>
+    <description>
+      The actual address the RPC server will bind to. If this optional address is
+      set, it overrides only the hostname portion of
+      dfs.federation.router.rpc-address. This is useful for making the name node
+      listen on all interfaces by setting it to 0.0.0.0.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.handler.count</name>
+    <value>10</value>
+    <description>
+      The number of server threads for the router to handle RPC requests from
+      clients.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.handler.queue.size</name>
+    <value>100</value>
+    <description>
+      The size of the queue for the number of handlers to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.reader.count</name>
+    <value>1</value>
+    <description>
+      The number of readers for the router to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.reader.queue.size</name>
+    <value>100</value>
+    <description>
+      The size of the queue for the number of readers for the router to handle RPC client requests.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.pool-size</name>
+    <value>1</value>
+    <description>
+      Size of the pool of connections from the router to namenodes.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.clean.ms</name>
+    <value>10000</value>
+    <description>
+      Time interval, in milliseconds, to check if the connection pool should
+      remove unused connections.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.connection.pool.clean.ms</name>
+    <value>60000</value>
+    <description>
+      Time interval, in milliseconds, to check if the connection manager should
+      remove unused connection pools.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.file.resolver.client.class</name>
     <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
index 8674682..bbb548ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/FederationTestUtils.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.federation;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.FileNotFoundException;
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
@@ -51,17 +52,11 @@ import org.apache.hadoop.security.AccessControlException;
  */
 public final class FederationTestUtils {
 
-  public final static String NAMESERVICE1 = "ns0";
-  public final static String NAMESERVICE2 = "ns1";
-  public final static String NAMENODE1 = "nn0";
-  public final static String NAMENODE2 = "nn1";
-  public final static String NAMENODE3 = "nn2";
-  public final static String NAMENODE4 = "nn3";
-  public final static String ROUTER1 = "router0";
-  public final static String ROUTER2 = "router1";
-  public final static String ROUTER3 = "router2";
-  public final static String ROUTER4 = "router3";
-  public final static long BLOCK_SIZE_BYTES = 134217728;
+  public final static String[] NAMESERVICES = {"ns0", "ns1"};
+  public final static String[] NAMENODES = {"nn0", "nn1", "nn2", "nn3"};
+  public final static String[] ROUTERS =
+      {"router0", "router1", "router2", "router3"};
+
 
   private FederationTestUtils() {
     // Utility class
@@ -81,7 +76,7 @@ public final class FederationTestUtils {
       triggeredException = e;
     }
     if (exceptionClass != null) {
-      assertNotNull("No exception was triggered, expected exception - "
+      assertNotNull("No exception was triggered, expected exception"
           + exceptionClass.getName(), triggeredException);
       assertEquals(exceptionClass, triggeredException.getClass());
     } else {
@@ -101,48 +96,43 @@ public final class FederationTestUtils {
       return report;
     }
     report.setHAServiceState(state);
-    report.setNamespaceInfo(new NamespaceInfo(1, "tesclusterid", ns, 0,
-            "testbuildvesion", "testsoftwareversion"));
+    NamespaceInfo nsInfo = new NamespaceInfo(
+        1, "tesclusterid", ns, 0, "testbuildvesion", "testsoftwareversion");
+    report.setNamespaceInfo(nsInfo);
     return report;
   }
 
   public static void waitNamenodeRegistered(ActiveNamenodeResolver resolver,
-      String nameserviceId, String namenodeId,
-      FederationNamenodeServiceState finalState)
+      String nsId, String nnId, FederationNamenodeServiceState finalState)
           throws InterruptedException, IllegalStateException, IOException {
 
     for (int loopCount = 0; loopCount < 20; loopCount++) {
-
       if (loopCount > 0) {
         Thread.sleep(1000);
       }
 
-      List<? extends FederationNamenodeContext> namenodes;
-      namenodes =
-          resolver.getNamenodesForNameserviceId(nameserviceId);
+      List<? extends FederationNamenodeContext> namenodes =
+          resolver.getNamenodesForNameserviceId(nsId);
       for (FederationNamenodeContext namenode : namenodes) {
-        if (namenodeId != null
-            && !namenode.getNamenodeId().equals(namenodeId)) {
-          // Keep looking
-          continue;
+        // Check if this is the Namenode we are checking
+        if (namenode.getNamenodeId() == nnId  ||
+            namenode.getNamenodeId().equals(nnId)) {
+          if (finalState != null && !namenode.getState().equals(finalState)) {
+            // Wrong state, wait a bit more
+            break;
+          } else {
+            // Found and verified
+            return;
+          }
         }
-        if (finalState != null && !namenode.getState().equals(finalState)) {
-          // Wrong state, wait a bit more
-          break;
-        }
-        // Found
-        return;
       }
     }
-    assertTrue("Failed to verify state store registration for state - "
-        + finalState + " - " + " - " + nameserviceId + " - ", false);
+    fail("Failed to verify State Store registration of " + nsId + " " + nnId +
+        " for state " + finalState);
   }
 
   public static boolean verifyDate(Date d1, Date d2, long precision) {
-    if (Math.abs(d1.getTime() - d2.getTime()) < precision) {
-      return true;
-    }
-    return false;
+    return Math.abs(d1.getTime() - d2.getTime()) < precision;
   }
 
   public static boolean addDirectory(FileSystem context, String path)
@@ -165,15 +155,14 @@ public final class FederationTestUtils {
     } catch (Exception e) {
       return false;
     }
-
     return false;
   }
 
-  public static boolean checkForFileInDirectory(FileSystem context,
-      String testPath, String targetFile) throws AccessControlException,
-          FileNotFoundException,
-          UnsupportedFileSystemException, IllegalArgumentException,
-          IOException {
+  public static boolean checkForFileInDirectory(
+      FileSystem context, String testPath, String targetFile)
+          throws IOException, AccessControlException, FileNotFoundException,
+          UnsupportedFileSystemException, IllegalArgumentException {
+
     FileStatus[] fileStatus = context.listStatus(new Path(testPath));
     String file = null;
     String verifyPath = testPath + "/" + targetFile;
@@ -194,7 +183,8 @@ public final class FederationTestUtils {
 
   public static int countContents(FileSystem context, String testPath)
       throws IOException {
-    FileStatus[] fileStatus = context.listStatus(new Path(testPath));
+    Path path = new Path(testPath);
+    FileStatus[] fileStatus = context.listStatus(path);
     return fileStatus.length;
   }
 
@@ -202,7 +192,7 @@ public final class FederationTestUtils {
       throws IOException {
     FsPermission permissions = new FsPermission("700");
     FSDataOutputStream writeStream = fs.create(new Path(path), permissions,
-        true, 1000, (short) 1, BLOCK_SIZE_BYTES, null);
+        true, 1000, (short) 1, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT, null);
     for (int i = 0; i < length; i++) {
       writeStream.write(i);
     }


---------------------------------------------------------------------
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: HDFS-12193. Fix style issues in HttpFS tests. Contributed by Zoran Dimitrijevic

Posted by in...@apache.org.
HDFS-12193. Fix style issues in HttpFS tests. Contributed by Zoran Dimitrijevic


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

Branch: refs/heads/HDFS-10467
Commit: c98201b5d83a700b4d081000065c6fd1a6ef2eed
Parents: 94ca52a
Author: Ravi Prakash <ra...@apache.org>
Authored: Mon Jul 24 19:06:15 2017 -0700
Committer: Ravi Prakash <ra...@apache.org>
Committed: Mon Jul 24 19:06:15 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/TestHttpFSServer.java | 106 +++++++++++--------
 .../fs/http/server/TestHttpFSServerNoACLs.java  |  15 +--
 .../http/server/TestHttpFSServerNoXAttrs.java   |  10 +-
 3 files changed, 77 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c98201b5/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
index 7cdb39c..0e1cc20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
@@ -73,6 +73,9 @@ import com.google.common.collect.Maps;
 import java.util.Properties;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 
+/**
+ * Main test class for HttpFSServer.
+ */
 public class TestHttpFSServer extends HFSTestCase {
 
   @Test
@@ -82,15 +85,20 @@ public class TestHttpFSServer extends HFSTestCase {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
 
     Configuration httpfsConf = new Configuration(false);
-    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir,
+                                                       httpfsConf);
     server.init();
     server.destroy();
   }
 
-  public static class MockGroups implements Service,Groups {
+  /**
+   * Mock groups.
+   */
+  public static class MockGroups implements Service, Groups {
 
     @Override
-    public void init(org.apache.hadoop.lib.server.Server server) throws ServiceException {
+    public void init(org.apache.hadoop.lib.server.Server server)
+        throws ServiceException {
     }
 
     @Override
@@ -112,8 +120,10 @@ public class TestHttpFSServer extends HFSTestCase {
     }
 
     @Override
-    public void serverStatusChange(org.apache.hadoop.lib.server.Server.Status oldStatus,
-                                   org.apache.hadoop.lib.server.Server.Status newStatus) throws ServiceException {
+    public void serverStatusChange(
+        org.apache.hadoop.lib.server.Server.Status oldStatus,
+        org.apache.hadoop.lib.server.Server.Status newStatus)
+        throws ServiceException {
     }
 
     @Override
@@ -300,25 +310,30 @@ public class TestHttpFSServer extends HFSTestCase {
     createHttpFSServer(false, false);
 
     URL url = new URL(TestJettyHelper.getJettyURL(),
-                      MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "nobody"));
+        MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
+                             "nobody"));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
+    Assert.assertEquals(conn.getResponseCode(),
+                        HttpURLConnection.HTTP_UNAUTHORIZED);
 
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
-                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
+        MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
+                             HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
-    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    BufferedReader reader = new BufferedReader(
+        new InputStreamReader(conn.getInputStream()));
     String line = reader.readLine();
     reader.close();
     Assert.assertTrue(line.contains("\"counters\":{"));
 
     url = new URL(TestJettyHelper.getJettyURL(),
-                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation",
-                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
+        MessageFormat.format(
+            "/webhdfs/v1/foo?user.name={0}&op=instrumentation",
+            HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
-    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+    Assert.assertEquals(conn.getResponseCode(),
+                        HttpURLConnection.HTTP_BAD_REQUEST);
   }
 
   @Test
@@ -330,10 +345,12 @@ public class TestHttpFSServer extends HFSTestCase {
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
-                      MessageFormat.format("/webhdfs/v1/?user.name={0}&op=liststatus", user));
+        MessageFormat.format("/webhdfs/v1/?user.name={0}&op=liststatus",
+                             user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
-    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    BufferedReader reader = new BufferedReader(
+        new InputStreamReader(conn.getInputStream()));
     reader.readLine();
     reader.close();
   }
@@ -369,10 +386,12 @@ public class TestHttpFSServer extends HFSTestCase {
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
-                      MessageFormat.format("/webhdfs/v1/tmp?user.name={0}&op=liststatus&filter=f*", user));
+        MessageFormat.format(
+            "/webhdfs/v1/tmp?user.name={0}&op=liststatus&filter=f*", user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
-    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    BufferedReader reader = new BufferedReader(
+        new InputStreamReader(conn.getInputStream()));
     reader.readLine();
     reader.close();
   }
@@ -384,15 +403,14 @@ public class TestHttpFSServer extends HFSTestCase {
    * @param perms The permission field, if any (may be null)
    * @throws Exception
    */
-  private void createWithHttp ( String filename, String perms )
-          throws Exception {
+  private void createWithHttp(String filename, String perms) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps;
-    if ( perms == null ) {
+    if (perms == null) {
       pathOps = MessageFormat.format(
               "/webhdfs/v1/{0}?user.name={1}&op=CREATE",
               filename, user);
@@ -422,7 +440,7 @@ public class TestHttpFSServer extends HFSTestCase {
           throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -449,7 +467,7 @@ public class TestHttpFSServer extends HFSTestCase {
                       String params) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -471,7 +489,7 @@ public class TestHttpFSServer extends HFSTestCase {
    * @return The value of 'permission' in statusJson
    * @throws Exception
    */
-  private String getPerms ( String statusJson ) throws Exception {
+  private String getPerms(String statusJson) throws Exception {
     JSONParser parser = new JSONParser();
     JSONObject jsonObject = (JSONObject) parser.parse(statusJson);
     JSONObject details = (JSONObject) jsonObject.get("FileStatus");
@@ -499,20 +517,20 @@ public class TestHttpFSServer extends HFSTestCase {
    * @return A List of Strings which are the elements of the ACL entries
    * @throws Exception
    */
-  private List<String> getAclEntries ( String statusJson ) throws Exception {
+  private List<String> getAclEntries(String statusJson) throws Exception {
     List<String> entries = new ArrayList<String>();
     JSONParser parser = new JSONParser();
     JSONObject jsonObject = (JSONObject) parser.parse(statusJson);
     JSONObject details = (JSONObject) jsonObject.get("AclStatus");
     JSONArray jsonEntries = (JSONArray) details.get("entries");
-    if ( jsonEntries != null ) {
+    if (jsonEntries != null) {
       for (Object e : jsonEntries) {
         entries.add(e.toString());
       }
     }
     return entries;
   }
-  
+
   /**
    * Parse xAttrs from JSON result of GETXATTRS call, return xAttrs Map.
    * @param statusJson JSON from GETXATTRS
@@ -533,8 +551,8 @@ public class TestHttpFSServer extends HFSTestCase {
     }
     return xAttrs;
   }
-  
-  /** Decode xattr value from string */
+
+  /** Decode xattr value from string. */
   private byte[] decodeXAttrValue(String value) throws IOException {
     if (value != null) {
       return XAttrCodec.decodeValue(value);
@@ -574,7 +592,7 @@ public class TestHttpFSServer extends HFSTestCase {
     statusJson = getStatus("/perm/p-321", "GETFILESTATUS");
     Assert.assertTrue("321".equals(getPerms(statusJson)));
   }
-  
+
   /**
    * Validate XAttr get/set/remove calls.
    */
@@ -594,12 +612,12 @@ public class TestHttpFSServer extends HFSTestCase {
 
     FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
     fs.mkdirs(new Path(dir));
-    
-    createWithHttp(path,null);
+
+    createWithHttp(path, null);
     String statusJson = getStatus(path, "GETXATTRS");
     Map<String, byte[]> xAttrs = getXAttrs(statusJson);
     Assert.assertEquals(0, xAttrs.size());
-    
+
     // Set two xattrs
     putCmd(path, "SETXATTR", setXAttrParam(name1, value1));
     putCmd(path, "SETXATTR", setXAttrParam(name2, value2));
@@ -608,25 +626,26 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(2, xAttrs.size());
     Assert.assertArrayEquals(value1, xAttrs.get(name1));
     Assert.assertArrayEquals(value2, xAttrs.get(name2));
-    
+
     // Remove one xattr
     putCmd(path, "REMOVEXATTR", "xattr.name=" + name1);
     statusJson = getStatus(path, "GETXATTRS");
     xAttrs = getXAttrs(statusJson);
     Assert.assertEquals(1, xAttrs.size());
     Assert.assertArrayEquals(value2, xAttrs.get(name2));
-    
+
     // Remove another xattr, then there is no xattr
     putCmd(path, "REMOVEXATTR", "xattr.name=" + name2);
     statusJson = getStatus(path, "GETXATTRS");
     xAttrs = getXAttrs(statusJson);
     Assert.assertEquals(0, xAttrs.size());
   }
-  
-  /** Params for setting an xAttr */
-  public static String setXAttrParam(String name, byte[] value) throws IOException {
+
+  /** Params for setting an xAttr. */
+  public static String setXAttrParam(String name, byte[] value)
+      throws IOException {
     return "xattr.name=" + name + "&xattr.value=" + XAttrCodec.encodeValue(
-        value, XAttrCodec.HEX) + "&encoding=hex&flag=create"; 
+        value, XAttrCodec.HEX) + "&encoding=hex&flag=create";
   }
 
   /**
@@ -791,7 +810,9 @@ public class TestHttpFSServer extends HFSTestCase {
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
-                      MessageFormat.format("/webhdfs/v1/tmp/foo?user.name={0}&op=open&offset=1&length=2", user));
+        MessageFormat.format(
+            "/webhdfs/v1/tmp/foo?user.name={0}&op=open&offset=1&length=2",
+            user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
     InputStream is = conn.getInputStream();
@@ -809,12 +830,13 @@ public class TestHttpFSServer extends HFSTestCase {
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
-                      MessageFormat.format("/webhdfs/v1/foo?user.name={0}", user));
+        MessageFormat.format("/webhdfs/v1/foo?user.name={0}", user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
     conn.setDoInput(true);
     conn.setDoOutput(true);
     conn.setRequestMethod("PUT");
-    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+    Assert.assertEquals(conn.getResponseCode(),
+        HttpURLConnection.HTTP_BAD_REQUEST);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c98201b5/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
index 289ddc4..c679dba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java
@@ -110,12 +110,12 @@ public class TestHttpFSServerNoACLs extends HTestCase {
 
     // HDFS configuration
     File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
-    if ( !hadoopConfDir.mkdirs() ) {
+    if (!hadoopConfDir.mkdirs()) {
       throw new IOException();
     }
 
     String fsDefaultName =
-            nnConf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
+        nnConf.get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
     Configuration conf = new Configuration(false);
     conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
 
@@ -146,7 +146,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
 
     ClassLoader cl = Thread.currentThread().getContextClassLoader();
     URL url = cl.getResource("webapp");
-    if ( url == null ) {
+    if (url == null) {
       throw new IOException();
     }
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
@@ -168,7 +168,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
           throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -179,7 +179,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
     conn.connect();
     int resp = conn.getResponseCode();
     BufferedReader reader;
-    if ( expectOK ) {
+    if (expectOK) {
       Assert.assertEquals(HttpURLConnection.HTTP_OK, resp);
       reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
       String res = reader.readLine();
@@ -204,7 +204,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
                       String params, boolean expectOK) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -216,7 +216,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
     conn.setRequestMethod("PUT");
     conn.connect();
     int resp = conn.getResponseCode();
-    if ( expectOK ) {
+    if (expectOK) {
       Assert.assertEquals(HttpURLConnection.HTTP_OK, resp);
     } else {
       Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp);
@@ -229,6 +229,7 @@ public class TestHttpFSServerNoACLs extends HTestCase {
   }
 
   /**
+   * Test without ACLs.
    * Ensure that
    * <ol>
    *   <li>GETFILESTATUS and LISTSTATUS work happily</li>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c98201b5/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
index 7571125..270989b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoXAttrs.java
@@ -111,7 +111,7 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
 
     // HDFS configuration
     File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
-    if ( !hadoopConfDir.mkdirs() ) {
+    if (!hadoopConfDir.mkdirs()) {
       throw new IOException();
     }
 
@@ -147,7 +147,7 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
 
     ClassLoader cl = Thread.currentThread().getContextClassLoader();
     URL url = cl.getResource("webapp");
-    if ( url == null ) {
+    if (url == null) {
       throw new IOException();
     }
     WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
@@ -168,7 +168,7 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
           throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -197,7 +197,7 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
                       String params) throws Exception {
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     // Remove leading / from filename
-    if ( filename.charAt(0) == '/' ) {
+    if (filename.charAt(0) == '/') {
       filename = filename.substring(1);
     }
     String pathOps = MessageFormat.format(
@@ -245,4 +245,4 @@ public class TestHttpFSServerNoXAttrs extends HTestCase {
     putCmd(path, "SETXATTR", TestHttpFSServer.setXAttrParam(name1, value1));
     putCmd(path, "REMOVEXATTR", "xattr.name=" + name1);
   }
-}
\ 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


[42/50] [abbrv] hadoop git commit: HDFS-11546. Federation Router RPC server. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
index ee6f57d..2875750 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.util.Time;
 
 /**
  * In-memory cache/mock of a namenode and file resolver. Stores the most
- * recently updated NN information for each nameservice and block pool. Also
+ * recently updated NN information for each nameservice and block pool. It also
  * stores a virtual mount table for resolving global namespace paths to local NN
  * paths.
  */
@@ -51,82 +51,93 @@ public class MockResolver
     implements ActiveNamenodeResolver, FileSubclusterResolver {
 
   private Map<String, List<? extends FederationNamenodeContext>> resolver =
-      new HashMap<String, List<? extends FederationNamenodeContext>>();
-  private Map<String, List<RemoteLocation>> locations =
-      new HashMap<String, List<RemoteLocation>>();
-  private Set<FederationNamespaceInfo> namespaces =
-      new HashSet<FederationNamespaceInfo>();
+      new HashMap<>();
+  private Map<String, List<RemoteLocation>> locations = new HashMap<>();
+  private Set<FederationNamespaceInfo> namespaces = new HashSet<>();
   private String defaultNamespace = null;
 
+
   public MockResolver(Configuration conf, StateStoreService store) {
     this.cleanRegistrations();
   }
 
-  public void addLocation(String mount, String nameservice, String location) {
-    RemoteLocation remoteLocation = new RemoteLocation(nameservice, location);
-    List<RemoteLocation> locationsList = locations.get(mount);
+  public void addLocation(String mount, String nsId, String location) {
+    List<RemoteLocation> locationsList = this.locations.get(mount);
     if (locationsList == null) {
-      locationsList = new LinkedList<RemoteLocation>();
-      locations.put(mount, locationsList);
+      locationsList = new LinkedList<>();
+      this.locations.put(mount, locationsList);
     }
+
+    final RemoteLocation remoteLocation = new RemoteLocation(nsId, location);
     if (!locationsList.contains(remoteLocation)) {
       locationsList.add(remoteLocation);
     }
 
     if (this.defaultNamespace == null) {
-      this.defaultNamespace = nameservice;
+      this.defaultNamespace = nsId;
     }
   }
 
   public synchronized void cleanRegistrations() {
-    this.resolver =
-        new HashMap<String, List<? extends FederationNamenodeContext>>();
-    this.namespaces = new HashSet<FederationNamespaceInfo>();
+    this.resolver = new HashMap<>();
+    this.namespaces = new HashSet<>();
   }
 
   @Override
   public void updateActiveNamenode(
-      String ns, InetSocketAddress successfulAddress) {
+      String nsId, InetSocketAddress successfulAddress) {
 
     String address = successfulAddress.getHostName() + ":" +
         successfulAddress.getPort();
-    String key = ns;
+    String key = nsId;
     if (key != null) {
       // Update the active entry
       @SuppressWarnings("unchecked")
-      List<FederationNamenodeContext> iterator =
-          (List<FederationNamenodeContext>) resolver.get(key);
-      for (FederationNamenodeContext namenode : iterator) {
+      List<FederationNamenodeContext> namenodes =
+          (List<FederationNamenodeContext>) this.resolver.get(key);
+      for (FederationNamenodeContext namenode : namenodes) {
         if (namenode.getRpcAddress().equals(address)) {
           MockNamenodeContext nn = (MockNamenodeContext) namenode;
           nn.setState(FederationNamenodeServiceState.ACTIVE);
           break;
         }
       }
-      Collections.sort(iterator, new NamenodePriorityComparator());
+      // This operation modifies the list so we need to be careful
+      synchronized(namenodes) {
+        Collections.sort(namenodes, new NamenodePriorityComparator());
+      }
     }
   }
 
   @Override
   public List<? extends FederationNamenodeContext>
       getNamenodesForNameserviceId(String nameserviceId) {
-    return resolver.get(nameserviceId);
+    // Return a copy of the list because it is updated periodically
+    List<? extends FederationNamenodeContext> namenodes =
+        this.resolver.get(nameserviceId);
+    return Collections.unmodifiableList(new ArrayList<>(namenodes));
   }
 
   @Override
   public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
       String blockPoolId) {
-    return resolver.get(blockPoolId);
+    // Return a copy of the list because it is updated periodically
+    List<? extends FederationNamenodeContext> namenodes =
+        this.resolver.get(blockPoolId);
+    return Collections.unmodifiableList(new ArrayList<>(namenodes));
   }
 
   private static class MockNamenodeContext
       implements FederationNamenodeContext {
+
+    private String namenodeId;
+    private String nameserviceId;
+
     private String webAddress;
     private String rpcAddress;
     private String serviceAddress;
     private String lifelineAddress;
-    private String namenodeId;
-    private String nameserviceId;
+
     private FederationNamenodeServiceState state;
     private long dateModified;
 
@@ -197,6 +208,7 @@ public class MockResolver
   @Override
   public synchronized boolean registerNamenode(NamenodeStatusReport report)
       throws IOException {
+
     MockNamenodeContext context = new MockNamenodeContext(
         report.getRpcAddress(), report.getServiceAddress(),
         report.getLifelineAddress(), report.getWebAddress(),
@@ -205,13 +217,14 @@ public class MockResolver
     String nsId = report.getNameserviceId();
     String bpId = report.getBlockPoolId();
     String cId = report.getClusterId();
+
     @SuppressWarnings("unchecked")
     List<MockNamenodeContext> existingItems =
-        (List<MockNamenodeContext>) resolver.get(nsId);
+        (List<MockNamenodeContext>) this.resolver.get(nsId);
     if (existingItems == null) {
-      existingItems = new ArrayList<MockNamenodeContext>();
-      resolver.put(bpId, existingItems);
-      resolver.put(nsId, existingItems);
+      existingItems = new ArrayList<>();
+      this.resolver.put(bpId, existingItems);
+      this.resolver.put(nsId, existingItems);
     }
     boolean added = false;
     for (int i=0; i<existingItems.size() && !added; i++) {
@@ -227,7 +240,7 @@ public class MockResolver
     Collections.sort(existingItems, new NamenodePriorityComparator());
 
     FederationNamespaceInfo info = new FederationNamespaceInfo(bpId, cId, nsId);
-    namespaces.add(info);
+    this.namespaces.add(info);
     return true;
   }
 
@@ -238,16 +251,13 @@ public class MockResolver
 
   @Override
   public PathLocation getDestinationForPath(String path) throws IOException {
-    String finalPath = null;
-    String nameservice = null;
-    Set<String> namespaceSet = new HashSet<String>();
-    LinkedList<RemoteLocation> remoteLocations =
-        new LinkedList<RemoteLocation>();
-    for(String key : this.locations.keySet()) {
-      if(path.startsWith(key)) {
+    Set<String> namespaceSet = new HashSet<>();
+    List<RemoteLocation> remoteLocations = new LinkedList<>();
+    for (String key : this.locations.keySet()) {
+      if (path.startsWith(key)) {
         for (RemoteLocation location : this.locations.get(key)) {
-          finalPath = location.getDest() + path.substring(key.length());
-          nameservice = location.getNameserviceId();
+          String finalPath = location.getDest() + path.substring(key.length());
+          String nameservice = location.getNameserviceId();
           RemoteLocation remoteLocation =
               new RemoteLocation(nameservice, finalPath);
           remoteLocations.add(remoteLocation);
@@ -265,7 +275,7 @@ public class MockResolver
 
   @Override
   public List<String> getMountPoints(String path) throws IOException {
-    List<String> mounts = new ArrayList<String>();
+    List<String> mounts = new ArrayList<>();
     if (path.equals("/")) {
       // Mounts only supported under root level
       for (String mount : this.locations.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
index 16d624c..39fcf7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /**
  * Constructs a router configuration with individual features enabled/disabled.
@@ -26,15 +27,32 @@ public class RouterConfigBuilder {
 
   private Configuration conf;
 
+  private boolean enableRpcServer = false;
+
   public RouterConfigBuilder(Configuration configuration) {
     this.conf = configuration;
   }
 
   public RouterConfigBuilder() {
-    this.conf = new Configuration();
+    this.conf = new Configuration(false);
+  }
+
+  public RouterConfigBuilder all() {
+    this.enableRpcServer = true;
+    return this;
+  }
+
+  public RouterConfigBuilder rpc(boolean enable) {
+    this.enableRpcServer = enable;
+    return this;
+  }
+
+  public RouterConfigBuilder rpc() {
+    return this.rpc(true);
   }
 
   public Configuration build() {
+    conf.setBoolean(DFSConfigKeys.DFS_ROUTER_RPC_ENABLE, this.enableRpcServer);
     return conf;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 55d04ad..4031b7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -17,27 +17,44 @@
  */
 package org.apache.hadoop.hdfs.server.federation;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_DEFAULT_NAMESERVICE;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.addDirectory;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.waitNamenodeRegistered;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -46,16 +63,49 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Test utility to mimic a federated HDFS cluster with a router.
+ * Test utility to mimic a federated HDFS cluster with multiple routers.
  */
 public class RouterDFSCluster {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterDFSCluster.class);
+
+  public static final String TEST_STRING = "teststring";
+  public static final String TEST_DIR = "testdir";
+  public static final String TEST_FILE = "testfile";
+
+
+  /** Nameservices in the federated cluster. */
+  private List<String> nameservices;
+  /** Namenodes in the federated cluster. */
+  private List<NamenodeContext> namenodes;
+  /** Routers in the federated cluster. */
+  private List<RouterContext> routers;
+  /** If the Namenodes are in high availability.*/
+  private boolean highAvailability;
+
+  /** Mini cluster. */
+  private MiniDFSCluster cluster;
+
+  /** Router configuration overrides. */
+  private Configuration routerOverrides;
+  /** Namenode configuration overrides. */
+  private Configuration namenodeOverrides;
+
+
   /**
    * Router context.
    */
@@ -69,13 +119,14 @@ public class RouterDFSCluster {
     private Configuration conf;
     private URI fileSystemUri;
 
-    public RouterContext(Configuration conf, String ns, String nn)
+    public RouterContext(Configuration conf, String nsId, String nnId)
         throws URISyntaxException {
-      this.namenodeId = nn;
-      this.nameserviceId = ns;
       this.conf = conf;
-      router = new Router();
-      router.init(conf);
+      this.nameserviceId = nsId;
+      this.namenodeId = nnId;
+
+      this.router = new Router();
+      this.router.init(conf);
     }
 
     public Router getRouter() {
@@ -99,18 +150,30 @@ public class RouterDFSCluster {
     }
 
     public void initRouter() throws URISyntaxException {
+      // Store the bound points for the router interfaces
+      InetSocketAddress rpcAddress = router.getRpcServerAddress();
+      if (rpcAddress != null) {
+        this.rpcPort = rpcAddress.getPort();
+        this.fileSystemUri =
+            URI.create("hdfs://" + NetUtils.getHostPortString(rpcAddress));
+        // Override the default FS to point to the router RPC
+        DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+        try {
+          this.fileContext = FileContext.getFileContext(conf);
+        } catch (UnsupportedFileSystemException e) {
+          this.fileContext = null;
+        }
+      }
     }
 
-    public DistributedFileSystem getFileSystem() throws IOException {
-      DistributedFileSystem fs =
-          (DistributedFileSystem) DistributedFileSystem.get(conf);
-      return fs;
+    public FileSystem getFileSystem() throws IOException {
+      return DistributedFileSystem.get(conf);
     }
 
     public DFSClient getClient(UserGroupInformation user)
         throws IOException, URISyntaxException, InterruptedException {
 
-      LOG.info("Connecting to router at " + fileSystemUri);
+      LOG.info("Connecting to router at {}", fileSystemUri);
       return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
         @Override
         public DFSClient run() throws IOException {
@@ -120,9 +183,8 @@ public class RouterDFSCluster {
     }
 
     public DFSClient getClient() throws IOException, URISyntaxException {
-
       if (client == null) {
-        LOG.info("Connecting to router at " + fileSystemUri);
+        LOG.info("Connecting to router at {}", fileSystemUri);
         client = new DFSClient(fileSystemUri, conf);
       }
       return client;
@@ -130,9 +192,10 @@ public class RouterDFSCluster {
   }
 
   /**
-   * Namenode context.
+   * Namenode context in the federated cluster.
    */
   public class NamenodeContext {
+    private Configuration conf;
     private NameNode namenode;
     private String nameserviceId;
     private String namenodeId;
@@ -143,14 +206,13 @@ public class RouterDFSCluster {
     private int httpPort;
     private URI fileSystemUri;
     private int index;
-    private Configuration conf;
     private DFSClient client;
 
-    public NamenodeContext(Configuration conf, String ns, String nn,
-        int index) {
+    public NamenodeContext(
+        Configuration conf, String nsId, String nnId, int index) {
       this.conf = conf;
-      this.namenodeId = nn;
-      this.nameserviceId = ns;
+      this.nameserviceId = nsId;
+      this.namenodeId = nnId;
       this.index = index;
     }
 
@@ -170,20 +232,19 @@ public class RouterDFSCluster {
       return this.fileContext;
     }
 
-    public void setNamenode(NameNode n) throws URISyntaxException {
-      namenode = n;
+    public void setNamenode(NameNode nn) throws URISyntaxException {
+      this.namenode = nn;
 
-      // Store the bound ports and override the default FS with the local NN's
-      // RPC
-      rpcPort = n.getNameNodeAddress().getPort();
-      servicePort = n.getServiceRpcAddress().getPort();
-      lifelinePort = n.getServiceRpcAddress().getPort();
-      httpPort = n.getHttpAddress().getPort();
-      fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
-      DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+      // Store the bound ports and override the default FS with the local NN RPC
+      this.rpcPort = nn.getNameNodeAddress().getPort();
+      this.servicePort = nn.getServiceRpcAddress().getPort();
+      this.lifelinePort = nn.getServiceRpcAddress().getPort();
+      this.httpPort = nn.getHttpAddress().getPort();
+      this.fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
+      DistributedFileSystem.setDefaultUri(this.conf, this.fileSystemUri);
 
       try {
-        this.fileContext = FileContext.getFileContext(conf);
+        this.fileContext = FileContext.getFileContext(this.conf);
       } catch (UnsupportedFileSystemException e) {
         this.fileContext = null;
       }
@@ -205,10 +266,8 @@ public class RouterDFSCluster {
       return namenode.getHttpAddress().getHostName() + ":" + httpPort;
     }
 
-    public DistributedFileSystem getFileSystem() throws IOException {
-      DistributedFileSystem fs =
-          (DistributedFileSystem) DistributedFileSystem.get(conf);
-      return fs;
+    public FileSystem getFileSystem() throws IOException {
+      return DistributedFileSystem.get(conf);
     }
 
     public void resetClient() {
@@ -218,7 +277,7 @@ public class RouterDFSCluster {
     public DFSClient getClient(UserGroupInformation user)
         throws IOException, URISyntaxException, InterruptedException {
 
-      LOG.info("Connecting to namenode at " + fileSystemUri);
+      LOG.info("Connecting to namenode at {}", fileSystemUri);
       return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
         @Override
         public DFSClient run() throws IOException {
@@ -229,7 +288,7 @@ public class RouterDFSCluster {
 
     public DFSClient getClient() throws IOException, URISyntaxException {
       if (client == null) {
-        LOG.info("Connecting to namenode at " + fileSystemUri);
+        LOG.info("Connecting to namenode at {}", fileSystemUri);
         client = new DFSClient(fileSystemUri, conf);
       }
       return client;
@@ -244,36 +303,20 @@ public class RouterDFSCluster {
     }
   }
 
-  public static final String NAMENODE1 = "nn0";
-  public static final String NAMENODE2 = "nn1";
-  public static final String NAMENODE3 = "nn2";
-  public static final String TEST_STRING = "teststring";
-  public static final String TEST_DIR = "testdir";
-  public static final String TEST_FILE = "testfile";
-
-  private List<String> nameservices;
-  private List<RouterContext> routers;
-  private List<NamenodeContext> namenodes;
-  private static final Log LOG = LogFactory.getLog(RouterDFSCluster.class);
-  private MiniDFSCluster cluster;
-  private boolean highAvailability;
-
-  protected static final int DEFAULT_HEARTBEAT_INTERVAL = 5;
-  protected static final int DEFAULT_CACHE_INTERVAL_SEC = 5;
-  private Configuration routerOverrides;
-  private Configuration namenodeOverrides;
-
-  private static final String NAMENODES = NAMENODE1 + "," + NAMENODE2;
-
-  public RouterDFSCluster(boolean ha, int numNameservices) {
-    this(ha, numNameservices, 2);
-  }
-
   public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
     this.highAvailability = ha;
     configureNameservices(numNameservices, numNamenodes);
   }
 
+  public RouterDFSCluster(boolean ha, int numNameservices) {
+    this(ha, numNameservices, 2);
+  }
+
+  /**
+   * Add configuration settings to override default Router settings.
+   *
+   * @param conf Router configuration overrides.
+   */
   public void addRouterOverrides(Configuration conf) {
     if (this.routerOverrides == null) {
       this.routerOverrides = conf;
@@ -282,6 +325,11 @@ public class RouterDFSCluster {
     }
   }
 
+  /**
+   * Add configuration settings to override default Namenode settings.
+   *
+   * @param conf Namenode configuration overrides.
+   */
   public void addNamenodeOverrides(Configuration conf) {
     if (this.namenodeOverrides == null) {
       this.namenodeOverrides = conf;
@@ -290,124 +338,134 @@ public class RouterDFSCluster {
     }
   }
 
-  public Configuration generateNamenodeConfiguration(
-      String defaultNameserviceId) {
-    Configuration c = new HdfsConfiguration();
+  /**
+   * Generate the configuration for a client.
+   *
+   * @param nsId Nameservice identifier.
+   * @return New namenode configuration.
+   */
+  public Configuration generateNamenodeConfiguration(String nsId) {
+    Configuration conf = new HdfsConfiguration();
 
-    c.set(DFSConfigKeys.DFS_NAMESERVICES, getNameservicesKey());
-    c.set("fs.defaultFS", "hdfs://" + defaultNameserviceId);
+    conf.set(DFS_NAMESERVICES, getNameservicesKey());
+    conf.set(FS_DEFAULT_NAME_KEY, "hdfs://" + nsId);
 
     for (String ns : nameservices) {
       if (highAvailability) {
-        c.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns, NAMENODES);
+        conf.set(
+            DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,
+            NAMENODES[0] + "," + NAMENODES[1]);
       }
 
       for (NamenodeContext context : getNamenodes(ns)) {
         String suffix = context.getConfSuffix();
 
-        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
             "127.0.0.1:" + context.rpcPort);
-        c.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
             "127.0.0.1:" + context.httpPort);
-        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
+        conf.set(DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
             "0.0.0.0");
       }
     }
 
-    if (namenodeOverrides != null) {
-      c.addResource(namenodeOverrides);
+    if (this.namenodeOverrides != null) {
+      conf.addResource(this.namenodeOverrides);
     }
-    return c;
+    return conf;
   }
 
+  /**
+   * Generate the configuration for a client.
+   *
+   * @return New configuration for a client.
+   */
   public Configuration generateClientConfiguration() {
-    Configuration conf = new HdfsConfiguration();
-    conf.addResource(generateNamenodeConfiguration(getNameservices().get(0)));
+    Configuration conf = new HdfsConfiguration(false);
+    String ns0 = getNameservices().get(0);
+    conf.addResource(generateNamenodeConfiguration(ns0));
     return conf;
   }
 
-  public Configuration generateRouterConfiguration(String localNameserviceId,
-      String localNamenodeId) throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    conf.addResource(generateNamenodeConfiguration(localNameserviceId));
+  /**
+   * Generate the configuration for a Router.
+   *
+   * @param nsId Nameservice identifier.
+   * @param nnId Namenode identifier.
+   * @return New configuration for a Router.
+   */
+  public Configuration generateRouterConfiguration(String nsId, String nnId) {
+
+    Configuration conf = new HdfsConfiguration(false);
+    conf.addResource(generateNamenodeConfiguration(nsId));
+
+    conf.setInt(DFS_ROUTER_HANDLER_COUNT_KEY, 10);
+    conf.set(DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
+
+    conf.set(DFS_ROUTER_DEFAULT_NAMESERVICE, nameservices.get(0));
 
     // Use mock resolver classes
-    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
-        MockResolver.class.getCanonicalName());
-    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
-        MockResolver.class.getCanonicalName());
+    conf.setClass(FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class, ActiveNamenodeResolver.class);
+    conf.setClass(FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class, FileSubclusterResolver.class);
 
     // Set the nameservice ID for the default NN monitor
-    conf.set(DFSConfigKeys.DFS_NAMESERVICE_ID, localNameserviceId);
-
-    if (localNamenodeId != null) {
-      conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, localNamenodeId);
+    conf.set(DFS_NAMESERVICE_ID, nsId);
+    if (nnId != null) {
+      conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
     }
 
-    StringBuilder routerBuilder = new StringBuilder();
-    for (String ns : nameservices) {
-      for (NamenodeContext context : getNamenodes(ns)) {
-        String suffix = context.getConfSuffix();
-
-        if (routerBuilder.length() != 0) {
-          routerBuilder.append(",");
-        }
-        routerBuilder.append(suffix);
+    // Add custom overrides if available
+    if (this.routerOverrides != null) {
+      for (Entry<String, String> entry : this.routerOverrides) {
+        String confKey = entry.getKey();
+        String confValue = entry.getValue();
+        conf.set(confKey, confValue);
       }
     }
-
     return conf;
   }
 
   public void configureNameservices(int numNameservices, int numNamenodes) {
-    nameservices = new ArrayList<String>();
-    for (int i = 0; i < numNameservices; i++) {
-      nameservices.add("ns" + i);
-    }
-    namenodes = new ArrayList<NamenodeContext>();
-    int index = 0;
-    for (String ns : nameservices) {
-      Configuration nnConf = generateNamenodeConfiguration(ns);
-      if (highAvailability) {
-        NamenodeContext context =
-            new NamenodeContext(nnConf, ns, NAMENODE1, index);
-        namenodes.add(context);
-        index++;
-
-        if (numNamenodes > 1) {
-          context = new NamenodeContext(nnConf, ns, NAMENODE2, index + 1);
-          namenodes.add(context);
-          index++;
-        }
+    this.nameservices = new ArrayList<>();
+    this.namenodes = new ArrayList<>();
 
-        if (numNamenodes > 2) {
-          context = new NamenodeContext(nnConf, ns, NAMENODE3, index + 1);
-          namenodes.add(context);
-          index++;
-        }
+    NamenodeContext context = null;
+    int nnIndex = 0;
+    for (int i=0; i<numNameservices; i++) {
+      String ns = "ns" + i;
+      this.nameservices.add("ns" + i);
 
+      Configuration nnConf = generateNamenodeConfiguration(ns);
+      if (!highAvailability) {
+        context = new NamenodeContext(nnConf, ns, null, nnIndex++);
+        this.namenodes.add(context);
       } else {
-        NamenodeContext context = new NamenodeContext(nnConf, ns, null, index);
-        namenodes.add(context);
-        index++;
+        for (int j=0; j<numNamenodes; j++) {
+          context = new NamenodeContext(nnConf, ns, NAMENODES[j], nnIndex++);
+          this.namenodes.add(context);
+        }
       }
     }
   }
 
   public String getNameservicesKey() {
-    StringBuilder ns = new StringBuilder();
-    for (int i = 0; i < nameservices.size(); i++) {
-      if (i > 0) {
-        ns.append(",");
+    StringBuilder sb = new StringBuilder();
+    for (String nsId : this.nameservices) {
+      if (sb.length() > 0) {
+        sb.append(",");
       }
-      ns.append(nameservices.get(i));
+      sb.append(nsId);
     }
-    return ns.toString();
+    return sb.toString();
   }
 
   public String getRandomNameservice() {
     Random r = new Random();
-    return nameservices.get(r.nextInt(nameservices.size()));
+    int randIndex = r.nextInt(nameservices.size());
+    return nameservices.get(randIndex);
   }
 
   public List<String> getNameservices() {
@@ -415,7 +473,7 @@ public class RouterDFSCluster {
   }
 
   public List<NamenodeContext> getNamenodes(String nameservice) {
-    ArrayList<NamenodeContext> nns = new ArrayList<NamenodeContext>();
+    List<NamenodeContext> nns = new ArrayList<>();
     for (NamenodeContext c : namenodes) {
       if (c.nameserviceId.equals(nameservice)) {
         nns.add(c);
@@ -426,23 +484,23 @@ public class RouterDFSCluster {
 
   public NamenodeContext getRandomNamenode() {
     Random rand = new Random();
-    return namenodes.get(rand.nextInt(namenodes.size()));
+    int i = rand.nextInt(this.namenodes.size());
+    return this.namenodes.get(i);
   }
 
   public List<NamenodeContext> getNamenodes() {
-    return namenodes;
+    return this.namenodes;
   }
 
   public boolean isHighAvailability() {
     return highAvailability;
   }
 
-  public NamenodeContext getNamenode(String nameservice,
-      String namenode) {
-    for (NamenodeContext c : namenodes) {
+  public NamenodeContext getNamenode(String nameservice, String namenode) {
+    for (NamenodeContext c : this.namenodes) {
       if (c.nameserviceId.equals(nameservice)) {
-        if (namenode == null || c.namenodeId == null || namenode.isEmpty()
-            || c.namenodeId.isEmpty()) {
+        if (namenode == null || namenode.isEmpty() ||
+            c.namenodeId == null || c.namenodeId.isEmpty()) {
           return c;
         } else if (c.namenodeId.equals(namenode)) {
           return c;
@@ -453,7 +511,7 @@ public class RouterDFSCluster {
   }
 
   public List<RouterContext> getRouters(String nameservice) {
-    ArrayList<RouterContext> nns = new ArrayList<RouterContext>();
+    List<RouterContext> nns = new ArrayList<>();
     for (RouterContext c : routers) {
       if (c.nameserviceId.equals(nameservice)) {
         nns.add(c);
@@ -462,14 +520,13 @@ public class RouterDFSCluster {
     return nns;
   }
 
-  public RouterContext getRouterContext(String nameservice,
-      String namenode) {
+  public RouterContext getRouterContext(String nsId, String nnId) {
     for (RouterContext c : routers) {
-      if (namenode == null) {
+      if (nnId == null) {
         return c;
       }
-      if (c.namenodeId.equals(namenode)
-          && c.nameserviceId.equals(nameservice)) {
+      if (c.namenodeId.equals(nnId) &&
+          c.nameserviceId.equals(nsId)) {
         return c;
       }
     }
@@ -485,10 +542,10 @@ public class RouterDFSCluster {
     return routers;
   }
 
-  public RouterContext buildRouter(String nameservice, String namenode)
+  public RouterContext buildRouter(String nsId, String nnId)
       throws URISyntaxException, IOException {
-    Configuration config = generateRouterConfiguration(nameservice, namenode);
-    RouterContext rc = new RouterContext(config, nameservice, namenode);
+    Configuration config = generateRouterConfiguration(nsId, nnId);
+    RouterContext rc = new RouterContext(config, nsId, nnId);
     return rc;
   }
 
@@ -500,10 +557,9 @@ public class RouterDFSCluster {
     try {
       MiniDFSNNTopology topology = new MiniDFSNNTopology();
       for (String ns : nameservices) {
-
         NSConf conf = new MiniDFSNNTopology.NSConf(ns);
         if (highAvailability) {
-          for(int i = 0; i < namenodes.size()/nameservices.size(); i++) {
+          for (int i=0; i<namenodes.size()/nameservices.size(); i++) {
             NNConf nnConf = new MiniDFSNNTopology.NNConf("nn" + i);
             conf.addNN(nnConf);
           }
@@ -516,11 +572,15 @@ public class RouterDFSCluster {
       topology.setFederation(true);
 
       // Start mini DFS cluster
-      Configuration nnConf = generateNamenodeConfiguration(nameservices.get(0));
+      String ns0 = nameservices.get(0);
+      Configuration nnConf = generateNamenodeConfiguration(ns0);
       if (overrideConf != null) {
         nnConf.addResource(overrideConf);
       }
-      cluster = new MiniDFSCluster.Builder(nnConf).nnTopology(topology).build();
+      cluster = new MiniDFSCluster.Builder(nnConf)
+          .numDataNodes(nameservices.size()*2)
+          .nnTopology(topology)
+          .build();
       cluster.waitActive();
 
       // Store NN pointers
@@ -530,28 +590,32 @@ public class RouterDFSCluster {
       }
 
     } catch (Exception e) {
-      LOG.error("Cannot start Router DFS cluster: " + e.getMessage(), e);
-      cluster.shutdown();
+      LOG.error("Cannot start Router DFS cluster: {}", e.getMessage(), e);
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 
   public void startRouters()
       throws InterruptedException, URISyntaxException, IOException {
-    // Create routers
-    routers = new ArrayList<RouterContext>();
-    for (String ns : nameservices) {
 
+    // Create one router per nameservice
+    this.routers = new ArrayList<>();
+    for (String ns : this.nameservices) {
       for (NamenodeContext context : getNamenodes(ns)) {
-        routers.add(buildRouter(ns, context.namenodeId));
+        RouterContext router = buildRouter(ns, context.namenodeId);
+        this.routers.add(router);
       }
     }
 
     // Start all routers
-    for (RouterContext router : routers) {
+    for (RouterContext router : this.routers) {
       router.router.start();
     }
+
     // Wait until all routers are active and record their ports
-    for (RouterContext router : routers) {
+    for (RouterContext router : this.routers) {
       waitActive(router);
       router.initRouter();
     }
@@ -570,22 +634,21 @@ public class RouterDFSCluster {
       }
       Thread.sleep(1000);
     }
-    assertFalse(
-        "Timeout waiting for " + router.router.toString() + " to activate.",
-        true);
+    fail("Timeout waiting for " + router.router + " to activate");
   }
 
-
   public void registerNamenodes() throws IOException {
-    for (RouterContext r : routers) {
+    for (RouterContext r : this.routers) {
       ActiveNamenodeResolver resolver = r.router.getNamenodeResolver();
-      for (NamenodeContext nn : namenodes) {
+      for (NamenodeContext nn : this.namenodes) {
         // Generate a report
-        NamenodeStatusReport report = new NamenodeStatusReport(nn.nameserviceId,
-            nn.namenodeId, nn.getRpcAddress(), nn.getServiceAddress(),
+        NamenodeStatusReport report = new NamenodeStatusReport(
+            nn.nameserviceId, nn.namenodeId,
+            nn.getRpcAddress(), nn.getServiceAddress(),
             nn.getLifelineAddress(), nn.getHttpAddress());
-        report.setNamespaceInfo(nn.namenode.getNamesystem().getFSImage()
-            .getStorage().getNamespaceInfo());
+        FSImage fsImage = nn.namenode.getNamesystem().getFSImage();
+        NamespaceInfo nsInfo = fsImage.getStorage().getNamespaceInfo();
+        report.setNamespaceInfo(nsInfo);
 
         // Determine HA state from nn public state string
         String nnState = nn.namenode.getState();
@@ -606,74 +669,97 @@ public class RouterDFSCluster {
 
   public void waitNamenodeRegistration()
       throws InterruptedException, IllegalStateException, IOException {
-    for (RouterContext r : routers) {
-      for (NamenodeContext nn : namenodes) {
-        FederationTestUtils.waitNamenodeRegistered(
-            r.router.getNamenodeResolver(), nn.nameserviceId, nn.namenodeId,
-            null);
+    for (RouterContext r : this.routers) {
+      Router router = r.router;
+      for (NamenodeContext nn : this.namenodes) {
+        ActiveNamenodeResolver nnResolver = router.getNamenodeResolver();
+        waitNamenodeRegistered(
+            nnResolver, nn.nameserviceId, nn.namenodeId, null);
       }
     }
   }
 
   public void waitRouterRegistrationQuorum(RouterContext router,
-      FederationNamenodeServiceState state, String nameservice, String namenode)
+      FederationNamenodeServiceState state, String nsId, String nnId)
           throws InterruptedException, IOException {
-    LOG.info("Waiting for NN - " + nameservice + ":" + namenode
-        + " to transition to state - " + state);
-    FederationTestUtils.waitNamenodeRegistered(
-        router.router.getNamenodeResolver(), nameservice, namenode, state);
+    LOG.info("Waiting for NN {} {} to transition to {}", nsId, nnId, state);
+    ActiveNamenodeResolver nnResolver = router.router.getNamenodeResolver();
+    waitNamenodeRegistered(nnResolver, nsId, nnId, state);
   }
 
-  public String getFederatedPathForNameservice(String ns) {
-    return "/" + ns;
+  /**
+   * Get the federated path for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Path in the Router.
+   */
+  public String getFederatedPathForNS(String nsId) {
+    return "/" + nsId;
   }
 
-  public String getNamenodePathForNameservice(String ns) {
-    return "/target-" + ns;
+  /**
+   * Get the namenode path for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Path in the Namenode.
+   */
+  public String getNamenodePathForNS(String nsId) {
+    return "/target-" + nsId;
   }
 
   /**
-   * @return example:
+   * Get the federated test directory for a nameservice.
+   * @param nsId Nameservice identifier.
+   * @return Example:
    *         <ul>
    *         <li>/ns0/testdir which maps to ns0->/target-ns0/testdir
    *         </ul>
    */
-  public String getFederatedTestDirectoryForNameservice(String ns) {
-    return getFederatedPathForNameservice(ns) + "/" + TEST_DIR;
+  public String getFederatedTestDirectoryForNS(String nsId) {
+    return getFederatedPathForNS(nsId) + "/" + TEST_DIR;
   }
 
   /**
+   * Get the namenode test directory for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/target-ns0/testdir
    *         </ul>
    */
-  public String getNamenodeTestDirectoryForNameservice(String ns) {
-    return getNamenodePathForNameservice(ns) + "/" + TEST_DIR;
+  public String getNamenodeTestDirectoryForNS(String nsId) {
+    return getNamenodePathForNS(nsId) + "/" + TEST_DIR;
   }
 
   /**
+   * Get the federated test file for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/ns0/testfile which maps to ns0->/target-ns0/testfile
    *         </ul>
    */
-  public String getFederatedTestFileForNameservice(String ns) {
-    return getFederatedPathForNameservice(ns) + "/" + TEST_FILE;
+  public String getFederatedTestFileForNS(String nsId) {
+    return getFederatedPathForNS(nsId) + "/" + TEST_FILE;
   }
 
   /**
+   * Get the namenode test file for a nameservice.
+   * @param nsId Nameservice identifier.
    * @return example:
    *         <ul>
    *         <li>/target-ns0/testfile
    *         </ul>
    */
-  public String getNamenodeTestFileForNameservice(String ns) {
-    return getNamenodePathForNameservice(ns) + "/" + TEST_FILE;
+  public String getNamenodeTestFileForNS(String nsId) {
+    return getNamenodePathForNS(nsId) + "/" + TEST_FILE;
   }
 
+  /**
+   * Stop the federated HDFS cluster.
+   */
   public void shutdown() {
-    cluster.shutdown();
+    if (cluster != null) {
+      cluster.shutdown();
+    }
     if (routers != null) {
       for (RouterContext context : routers) {
         stopRouter(context);
@@ -681,9 +767,12 @@ public class RouterDFSCluster {
     }
   }
 
+  /**
+   * Stop a router.
+   * @param router Router context.
+   */
   public void stopRouter(RouterContext router) {
     try {
-
       router.router.shutDown();
 
       int loopCount = 0;
@@ -691,7 +780,7 @@ public class RouterDFSCluster {
         loopCount++;
         Thread.sleep(1000);
         if (loopCount > 20) {
-          LOG.error("Unable to shutdown router - " + router.rpcPort);
+          LOG.error("Cannot shutdown router {}", router.rpcPort);
           break;
         }
       }
@@ -714,26 +803,28 @@ public class RouterDFSCluster {
     for (String ns : getNameservices()) {
       NamenodeContext context = getNamenode(ns, null);
       if (!createTestDirectoriesNamenode(context)) {
-        throw new IOException("Unable to create test directory for ns - " + ns);
+        throw new IOException("Cannot create test directory for ns " + ns);
       }
     }
   }
 
   public boolean createTestDirectoriesNamenode(NamenodeContext nn)
       throws IOException {
-    return FederationTestUtils.addDirectory(nn.getFileSystem(),
-        getNamenodeTestDirectoryForNameservice(nn.nameserviceId));
+    FileSystem fs = nn.getFileSystem();
+    String testDir = getNamenodeTestDirectoryForNS(nn.nameserviceId);
+    return addDirectory(fs, testDir);
   }
 
   public void deleteAllFiles() throws IOException {
     // Delete all files via the NNs and verify
     for (NamenodeContext context : getNamenodes()) {
-      FileStatus[] status = context.getFileSystem().listStatus(new Path("/"));
-      for(int i = 0; i <status.length; i++) {
+      FileSystem fs = context.getFileSystem();
+      FileStatus[] status = fs.listStatus(new Path("/"));
+      for (int i = 0; i <status.length; i++) {
         Path p = status[i].getPath();
-        context.getFileSystem().delete(p, true);
+        fs.delete(p, true);
       }
-      status = context.getFileSystem().listStatus(new Path("/"));
+      status = fs.listStatus(new Path("/"));
       assertEquals(status.length, 0);
     }
   }
@@ -754,14 +845,34 @@ public class RouterDFSCluster {
       MockResolver resolver =
           (MockResolver) r.router.getSubclusterResolver();
       // create table entries
-      for (String ns : nameservices) {
+      for (String nsId : nameservices) {
         // Direct path
-        resolver.addLocation(getFederatedPathForNameservice(ns), ns,
-            getNamenodePathForNameservice(ns));
+        String routerPath = getFederatedPathForNS(nsId);
+        String nnPath = getNamenodePathForNS(nsId);
+        resolver.addLocation(routerPath, nsId, nnPath);
       }
 
-      // Root path goes to both NS1
-      resolver.addLocation("/", nameservices.get(0), "/");
+      // Root path points to both first nameservice
+      String ns0 = nameservices.get(0);
+      resolver.addLocation("/", ns0, "/");
+    }
+  }
+
+  public MiniDFSCluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * Wait until the federated cluster is up and ready.
+   * @throws IOException If we cannot wait for the cluster to be up.
+   */
+  public void waitClusterUp() throws IOException {
+    cluster.waitClusterUp();
+    registerNamenodes();
+    try {
+      waitNamenodeRegistration();
+    } catch (Exception e) {
+      throw new IOException("Cannot wait for the namenodes", e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
index 8c720c7..d8afb39 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.IOException;
 import java.net.URISyntaxException;
@@ -51,6 +52,10 @@ public class TestRouter {
     conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
         MockResolver.class.getCanonicalName());
 
+    // Bind to any available port
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_BIND_HOST_KEY, "0.0.0.0");
+    conf.set(DFSConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "127.0.0.1:0");
+
     // Simulate a co-located NN
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns0");
     conf.set("fs.defaultFS", "hdfs://" + "ns0");
@@ -90,7 +95,31 @@ public class TestRouter {
   @Test
   public void testRouterService() throws InterruptedException, IOException {
 
+    // Rpc only
+    testRouterStartup(new RouterConfigBuilder(conf).rpc().build());
+
     // Run with all services
-    testRouterStartup((new RouterConfigBuilder(conf)).build());
+    testRouterStartup(new RouterConfigBuilder(conf).all().build());
+  }
+
+  @Test
+  public void testRouterRestartRpcService() throws IOException {
+
+    // Start
+    Router router = new Router();
+    router.init(new RouterConfigBuilder(conf).rpc().build());
+    router.start();
+
+    // Verify RPC server is running
+    assertNotNull(router.getRpcServerAddress());
+    RouterRpcServer rpcServer = router.getRpcServer();
+    assertNotNull(rpcServer);
+    assertEquals(STATE.STARTED, rpcServer.getServiceState());
+
+    // Stop router and RPC server
+    router.stop();
+    assertEquals(STATE.STOPPED, rpcServer.getServiceState());
+
+    router.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
new file mode 100644
index 0000000..af506c9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
@@ -0,0 +1,869 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.addDirectory;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.countContents;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.createFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.deleteFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.getFileStatus;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
+import static org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.TEST_STRING;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service.STATE;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The the RPC interface of the {@link Router} implemented by
+ * {@link RouterRpcServer}.
+ */
+public class TestRouterRpc {
+
+  /** Federated HDFS cluster. */
+  private static RouterDFSCluster cluster;
+
+  /** Random Router for this federated cluster. */
+  private RouterContext router;
+
+  /** Random nameservice in the federated cluster.  */
+  private String ns;
+  /** First namenode in the nameservice. */
+  private NamenodeContext namenode;
+
+  /** Client interface to the Router. */
+  private ClientProtocol routerProtocol;
+  /** Client interface to the Namenode. */
+  private ClientProtocol nnProtocol;
+
+  /** Filesystem interface to the Router. */
+  private FileSystem routerFS;
+  /** Filesystem interface to the Namenode. */
+  private FileSystem nnFS;
+
+  /** File in the Router. */
+  private String routerFile;
+  /** File in the Namenode. */
+  private String nnFile;
+
+
+  @BeforeClass
+  public static void globalSetUp() throws Exception {
+    cluster = new RouterDFSCluster(false, 2);
+
+    // Start NNs and DNs and wait until ready
+    cluster.startCluster();
+
+    // Start routers with only an RPC service
+    cluster.addRouterOverrides((new RouterConfigBuilder()).rpc().build());
+    cluster.startRouters();
+
+    // Register and verify all NNs with all routers
+    cluster.registerNamenodes();
+    cluster.waitNamenodeRegistration();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    cluster.shutdown();
+  }
+
+  @Before
+  public void testSetup() throws Exception {
+
+    // Create mock locations
+    cluster.installMockLocations();
+
+    // Delete all files via the NNs and verify
+    cluster.deleteAllFiles();
+
+    // Create test fixtures on NN
+    cluster.createTestDirectoriesNamenode();
+
+    // Wait to ensure NN has fully created its test directories
+    Thread.sleep(100);
+
+    // Pick a NS, namenode and router for this test
+    this.router = cluster.getRandomRouter();
+    this.ns = cluster.getRandomNameservice();
+    this.namenode = cluster.getNamenode(ns, null);
+
+    // Handles to the ClientProtocol interface
+    this.routerProtocol = router.getClient().getNamenode();
+    this.nnProtocol = namenode.getClient().getNamenode();
+
+    // Handles to the filesystem client
+    this.nnFS = namenode.getFileSystem();
+    this.routerFS = router.getFileSystem();
+
+    // Create a test file on the NN
+    Random r = new Random();
+    String randomFile = "testfile-" + r.nextInt();
+    this.nnFile =
+        cluster.getNamenodeTestDirectoryForNS(ns) + "/" + randomFile;
+    this.routerFile =
+        cluster.getFederatedTestDirectoryForNS(ns) + "/" + randomFile;
+
+    createFile(nnFS, nnFile, 32);
+    verifyFileExists(nnFS, nnFile);
+  }
+
+  @Test
+  public void testRpcService() throws IOException {
+    Router testRouter = new Router();
+    List<String> nss = cluster.getNameservices();
+    String ns0 = nss.get(0);
+    Configuration routerConfig = cluster.generateRouterConfiguration(ns0, null);
+    RouterRpcServer server = new RouterRpcServer(routerConfig, testRouter,
+        testRouter.getNamenodeResolver(), testRouter.getSubclusterResolver());
+    server.init(routerConfig);
+    assertEquals(STATE.INITED, server.getServiceState());
+    server.start();
+    assertEquals(STATE.STARTED, server.getServiceState());
+    server.stop();
+    assertEquals(STATE.STOPPED, server.getServiceState());
+    server.close();
+    testRouter.close();
+  }
+
+  protected RouterDFSCluster getCluster() {
+    return TestRouterRpc.cluster;
+  }
+
+  protected RouterContext getRouterContext() {
+    return this.router;
+  }
+
+  protected void setRouter(RouterContext r)
+      throws IOException, URISyntaxException {
+    this.router = r;
+    this.routerProtocol = r.getClient().getNamenode();
+    this.routerFS = r.getFileSystem();
+  }
+
+  protected FileSystem getRouterFileSystem() {
+    return this.routerFS;
+  }
+
+  protected FileSystem getNamenodeFileSystem() {
+    return this.nnFS;
+  }
+
+  protected ClientProtocol getRouterProtocol() {
+    return this.routerProtocol;
+  }
+
+  protected ClientProtocol getNamenodeProtocol() {
+    return this.nnProtocol;
+  }
+
+  protected NamenodeContext getNamenode() {
+    return this.namenode;
+  }
+
+  protected void setNamenodeFile(String filename) {
+    this.nnFile = filename;
+  }
+
+  protected String getNamenodeFile() {
+    return this.nnFile;
+  }
+
+  protected void setRouterFile(String filename) {
+    this.routerFile = filename;
+  }
+
+  protected String getRouterFile() {
+    return this.routerFile;
+  }
+
+  protected void setNamenode(NamenodeContext nn)
+      throws IOException, URISyntaxException {
+    this.namenode = nn;
+    this.nnProtocol = nn.getClient().getNamenode();
+    this.nnFS = nn.getFileSystem();
+  }
+
+  protected String getNs() {
+    return this.ns;
+  }
+
+  protected void setNs(String nameservice) {
+    this.ns = nameservice;
+  }
+
+  protected static void compareResponses(
+      ClientProtocol protocol1, ClientProtocol protocol2,
+      Method m, Object[] paramList) {
+
+    Object return1 = null;
+    Exception exception1 = null;
+    try {
+      return1 = m.invoke(protocol1, paramList);
+    } catch (Exception ex) {
+      exception1 = ex;
+    }
+
+    Object return2 = null;
+    Exception exception2 = null;
+    try {
+      return2 = m.invoke(protocol2, paramList);
+    } catch (Exception ex) {
+      exception2 = ex;
+    }
+
+    assertEquals(return1, return2);
+    if (exception1 == null && exception2 == null) {
+      return;
+    }
+
+    assertEquals(
+        exception1.getCause().getClass(),
+        exception2.getCause().getClass());
+  }
+
+  @Test
+  public void testProxyListFiles() throws IOException, InterruptedException,
+      URISyntaxException, NoSuchMethodException, SecurityException {
+
+    // Verify that the root listing is a union of the mount table destinations
+    // and the files stored at all nameservices mounted at the root (ns0 + ns1)
+    //
+    // / -->
+    // /ns0 (from mount table)
+    // /ns1 (from mount table)
+    // all items in / of ns0 (default NS)
+
+    // Collect the mount table entries from the root mount point
+    Set<String> requiredPaths = new TreeSet<>();
+    FileSubclusterResolver fileResolver =
+        router.getRouter().getSubclusterResolver();
+    for (String mount : fileResolver.getMountPoints("/")) {
+      requiredPaths.add(mount);
+    }
+
+    // Collect all files/dirs on the root path of the default NS
+    String defaultNs = cluster.getNameservices().get(0);
+    NamenodeContext nn = cluster.getNamenode(defaultNs, null);
+    FileStatus[] iterator = nn.getFileSystem().listStatus(new Path("/"));
+    for (FileStatus file : iterator) {
+      requiredPaths.add(file.getPath().getName());
+    }
+
+    // Fetch listing
+    DirectoryListing listing =
+        routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
+    Iterator<String> requiredPathsIterator = requiredPaths.iterator();
+    // Match each path returned and verify order returned
+    for(HdfsFileStatus f : listing.getPartialListing()) {
+      String fileName = requiredPathsIterator.next();
+      String currentFile = f.getFullPath(new Path("/")).getName();
+      assertEquals(currentFile, fileName);
+    }
+
+    // Verify the total number of results found/matched
+    assertEquals(requiredPaths.size(), listing.getPartialListing().length);
+
+    // List a path that doesn't exist and validate error response with NN
+    // behavior.
+    Method m = ClientProtocol.class.getMethod(
+        "getListing", String.class, byte[].class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, HdfsFileStatus.EMPTY_NAME, false});
+  }
+
+  @Test
+  public void testProxyListFilesWithConflict()
+      throws IOException, InterruptedException {
+
+    // Add a directory to the namespace that conflicts with a mount point
+    NamenodeContext nn = cluster.getNamenode(ns, null);
+    FileSystem nnFs = nn.getFileSystem();
+    addDirectory(nnFs, cluster.getFederatedTestDirectoryForNS(ns));
+
+    FileSystem routerFs = router.getFileSystem();
+    int initialCount = countContents(routerFs, "/");
+
+    // Root file system now for NS X:
+    // / ->
+    // /ns0 (mount table)
+    // /ns1 (mount table)
+    // /target-ns0 (the target folder for the NS0 mapped to /
+    // /nsX (local directory that duplicates mount table)
+    int newCount = countContents(routerFs, "/");
+    assertEquals(initialCount, newCount);
+
+    // Verify that each root path is readable and contains one test directory
+    assertEquals(1, countContents(routerFs, cluster.getFederatedPathForNS(ns)));
+
+    // Verify that real folder for the ns contains a single test directory
+    assertEquals(1, countContents(nnFs, cluster.getNamenodePathForNS(ns)));
+
+  }
+
+  protected void testRename(RouterContext testRouter, String filename,
+      String renamedFile, boolean exceptionExpected) throws IOException {
+
+    createFile(testRouter.getFileSystem(), filename, 32);
+    // verify
+    verifyFileExists(testRouter.getFileSystem(), filename);
+    // rename
+    boolean exceptionThrown = false;
+    try {
+      DFSClient client = testRouter.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      clientProtocol.rename(filename, renamedFile);
+    } catch (Exception ex) {
+      exceptionThrown = true;
+    }
+    if (exceptionExpected) {
+      // Error was expected
+      assertTrue(exceptionThrown);
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(filename), true));
+    } else {
+      // No error was expected
+      assertFalse(exceptionThrown);
+      // verify
+      assertTrue(verifyFileExists(testRouter.getFileSystem(), renamedFile));
+      // delete
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(renamedFile), true));
+    }
+  }
+
+  protected void testRename2(RouterContext testRouter, String filename,
+      String renamedFile, boolean exceptionExpected) throws IOException {
+    createFile(testRouter.getFileSystem(), filename, 32);
+    // verify
+    verifyFileExists(testRouter.getFileSystem(), filename);
+    // rename
+    boolean exceptionThrown = false;
+    try {
+      DFSClient client = testRouter.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      clientProtocol.rename2(filename, renamedFile, new Options.Rename[] {});
+    } catch (Exception ex) {
+      exceptionThrown = true;
+    }
+    assertEquals(exceptionExpected, exceptionThrown);
+    if (exceptionExpected) {
+      // Error was expected
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(filename), true));
+    } else {
+      // verify
+      assertTrue(verifyFileExists(testRouter.getFileSystem(), renamedFile));
+      // delete
+      FileContext fileContext = testRouter.getFileContext();
+      assertTrue(fileContext.delete(new Path(renamedFile), true));
+    }
+  }
+
+  @Test
+  public void testProxyRenameFiles() throws IOException, InterruptedException {
+
+    Thread.sleep(5000);
+    List<String> nss = cluster.getNameservices();
+    String ns0 = nss.get(0);
+    String ns1 = nss.get(1);
+
+    // Rename within the same namespace
+    // /ns0/testdir/testrename -> /ns0/testdir/testrename-append
+    String filename =
+        cluster.getFederatedTestDirectoryForNS(ns0) + "/testrename";
+    String renamedFile = filename + "-append";
+    testRename(router, filename, renamedFile, false);
+    testRename2(router, filename, renamedFile, false);
+
+    // Rename a file to a destination that is in a different namespace (fails)
+    filename = cluster.getFederatedTestDirectoryForNS(ns0) + "/testrename";
+    renamedFile = cluster.getFederatedTestDirectoryForNS(ns1) + "/testrename";
+    testRename(router, filename, renamedFile, true);
+    testRename2(router, filename, renamedFile, true);
+  }
+
+  @Test
+  public void testProxyChownFiles() throws Exception {
+
+    String newUsername = "TestUser";
+    String newGroup = "TestGroup";
+
+    // change owner
+    routerProtocol.setOwner(routerFile, newUsername, newGroup);
+
+    // Verify with NN
+    FileStatus file = getFileStatus(namenode.getFileSystem(), nnFile);
+    assertEquals(file.getOwner(), newUsername);
+    assertEquals(file.getGroup(), newGroup);
+
+    // Bad request and validate router response matches NN response.
+    Method m = ClientProtocol.class.getMethod("setOwner", String.class,
+        String.class, String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, newUsername, newGroup});
+  }
+
+  @Test
+  public void testProxyGetStats() throws Exception {
+
+    long[] combinedData = routerProtocol.getStats();
+
+    long[] individualData = new long[10];
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      long[] data = clientProtocol.getStats();
+      for (int i = 0; i < data.length; i++) {
+        individualData[i] += data[i];
+      }
+      assert(data.length == combinedData.length);
+    }
+
+    for (int i = 0; i < combinedData.length && i < individualData.length; i++) {
+      if (i == ClientProtocol.GET_STATS_REMAINING_IDX) {
+        // Skip available storage as this fluctuates in mini cluster
+        continue;
+      }
+      assertEquals(combinedData[i], individualData[i]);
+    }
+  }
+
+  @Test
+  public void testProxyGetDatanodeReport() throws Exception {
+
+    DatanodeInfo[] combinedData =
+        routerProtocol.getDatanodeReport(DatanodeReportType.ALL);
+
+    Set<Integer> individualData = new HashSet<Integer>();
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      DatanodeInfo[] data =
+          clientProtocol.getDatanodeReport(DatanodeReportType.ALL);
+      for (int i = 0; i < data.length; i++) {
+        // Collect unique DNs based on their xfer port
+        DatanodeInfo info = data[i];
+        individualData.add(info.getXferPort());
+      }
+    }
+    assertEquals(combinedData.length, individualData.size());
+  }
+
+  @Test
+  public void testProxyGetDatanodeStorageReport()
+      throws IOException, InterruptedException, URISyntaxException {
+
+    DatanodeStorageReport[] combinedData =
+        routerProtocol.getDatanodeStorageReport(DatanodeReportType.ALL);
+
+    Set<String> individualData = new HashSet<>();
+    for (String nameservice : cluster.getNameservices()) {
+      NamenodeContext n = cluster.getNamenode(nameservice, null);
+      DFSClient client = n.getClient();
+      ClientProtocol clientProtocol = client.getNamenode();
+      DatanodeStorageReport[] data =
+          clientProtocol.getDatanodeStorageReport(DatanodeReportType.ALL);
+      for (DatanodeStorageReport report : data) {
+        // Determine unique DN instances
+        DatanodeInfo dn = report.getDatanodeInfo();
+        individualData.add(dn.toString());
+      }
+    }
+    assertEquals(combinedData.length, individualData.size());
+  }
+
+  @Test
+  public void testProxyMkdir() throws Exception {
+
+    // Check the initial folders
+    FileStatus[] filesInitial = routerFS.listStatus(new Path("/"));
+
+    // Create a directory via the router at the root level
+    String dirPath = "/testdir";
+    FsPermission permission = new FsPermission("705");
+    routerProtocol.mkdirs(dirPath, permission, false);
+
+    // Verify the root listing has the item via the router
+    FileStatus[] files = routerFS.listStatus(new Path("/"));
+    assertEquals(Arrays.toString(files) + " should be " +
+        Arrays.toString(filesInitial) + " + " + dirPath,
+        filesInitial.length + 1, files.length);
+    assertTrue(verifyFileExists(routerFS, dirPath));
+
+    // Verify the directory is present in only 1 Namenode
+    int foundCount = 0;
+    for (NamenodeContext n : cluster.getNamenodes()) {
+      if (verifyFileExists(n.getFileSystem(), dirPath)) {
+        foundCount++;
+      }
+    }
+    assertEquals(1, foundCount);
+    assertTrue(deleteFile(routerFS, dirPath));
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("mkdirs", String.class,
+        FsPermission.class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, permission, false});
+  }
+
+  @Test
+  public void testProxyChmodFiles() throws Exception {
+
+    FsPermission permission = new FsPermission("444");
+
+    // change permissions
+    routerProtocol.setPermission(routerFile, permission);
+
+    // Validate permissions NN
+    FileStatus file = getFileStatus(namenode.getFileSystem(), nnFile);
+    assertEquals(permission, file.getPermission());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "setPermission", String.class, FsPermission.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, permission});
+  }
+
+  @Test
+  public void testProxySetReplication() throws Exception {
+
+    // Check current replication via NN
+    FileStatus file = getFileStatus(nnFS, nnFile);
+    assertEquals(1, file.getReplication());
+
+    // increment replication via router
+    routerProtocol.setReplication(routerFile, (short) 2);
+
+    // Verify via NN
+    file = getFileStatus(nnFS, nnFile);
+    assertEquals(2, file.getReplication());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "setReplication", String.class, short.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, (short) 2});
+  }
+
+  @Test
+  public void testProxyTruncateFile() throws Exception {
+
+    // Check file size via NN
+    FileStatus file = getFileStatus(nnFS, nnFile);
+    assertTrue(file.getLen() > 0);
+
+    // Truncate to 0 bytes via router
+    routerProtocol.truncate(routerFile, 0, "testclient");
+
+    // Verify via NN
+    file = getFileStatus(nnFS, nnFile);
+    assertEquals(0, file.getLen());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "truncate", String.class, long.class, String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, (long) 0, "testclient"});
+  }
+
+  @Test
+  public void testProxyGetBlockLocations() throws Exception {
+
+    // Fetch block locations via router
+    LocatedBlocks locations =
+        routerProtocol.getBlockLocations(routerFile, 0, 1024);
+    assertEquals(1, locations.getLocatedBlocks().size());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "getBlockLocations", String.class, long.class, long.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol,
+        m, new Object[] {badPath, (long) 0, (long) 0});
+  }
+
+  @Test
+  public void testProxyStoragePolicy() throws Exception {
+
+    // Query initial policy via NN
+    HdfsFileStatus status = namenode.getClient().getFileInfo(nnFile);
+
+    // Set a random policy via router
+    BlockStoragePolicy[] policies = namenode.getClient().getStoragePolicies();
+    BlockStoragePolicy policy = policies[0];
+
+    while (policy.isCopyOnCreateFile()) {
+      // Pick a non copy on create policy
+      Random rand = new Random();
+      int randIndex = rand.nextInt(policies.length);
+      policy = policies[randIndex];
+    }
+    routerProtocol.setStoragePolicy(routerFile, policy.getName());
+
+    // Verify policy via NN
+    HdfsFileStatus newStatus = namenode.getClient().getFileInfo(nnFile);
+    assertTrue(newStatus.getStoragePolicy() == policy.getId());
+    assertTrue(newStatus.getStoragePolicy() != status.getStoragePolicy());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("setStoragePolicy", String.class,
+        String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol,
+        m, new Object[] {badPath, "badpolicy"});
+  }
+
+  @Test
+  public void testProxyGetPreferedBlockSize() throws Exception {
+
+    // Query via NN and Router and verify
+    long namenodeSize = nnProtocol.getPreferredBlockSize(nnFile);
+    long routerSize = routerProtocol.getPreferredBlockSize(routerFile);
+    assertEquals(routerSize, namenodeSize);
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "getPreferredBlockSize", String.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(
+        routerProtocol, nnProtocol, m, new Object[] {badPath});
+  }
+
+  private void testConcat(
+      String source, String target, boolean failureExpected) {
+    boolean failure = false;
+    try {
+      // Concat test file with fill block length file via router
+      routerProtocol.concat(target, new String[] {source});
+    } catch (IOException ex) {
+      failure = true;
+    }
+    assertEquals(failureExpected, failure);
+  }
+
+  @Test
+  public void testProxyConcatFile() throws Exception {
+
+    // Create a stub file in the primary ns
+    String sameNameservice = ns;
+    String existingFile =
+        cluster.getFederatedTestDirectoryForNS(sameNameservice) +
+        "_concatfile";
+    int existingFileSize = 32;
+    createFile(routerFS, existingFile, existingFileSize);
+
+    // Identify an alternate nameservice that doesn't match the existing file
+    String alternateNameservice = null;
+    for (String n : cluster.getNameservices()) {
+      if (!n.equals(sameNameservice)) {
+        alternateNameservice = n;
+        break;
+      }
+    }
+
+    // Create new files, must be a full block to use concat. One file is in the
+    // same namespace as the target file, the other is in a different namespace.
+    String altRouterFile =
+        cluster.getFederatedTestDirectoryForNS(alternateNameservice) +
+        "_newfile";
+    String sameRouterFile =
+        cluster.getFederatedTestDirectoryForNS(sameNameservice) +
+        "_newfile";
+    createFile(routerFS, altRouterFile, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
+    createFile(routerFS, sameRouterFile, DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
+
+    // Concat in different namespaces, fails
+    testConcat(existingFile, altRouterFile, true);
+
+    // Concat in same namespaces, succeeds
+    testConcat(existingFile, sameRouterFile, false);
+
+    // Check target file length
+    FileStatus status = getFileStatus(routerFS, sameRouterFile);
+    assertEquals(
+        existingFileSize + DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT,
+        status.getLen());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod(
+        "concat", String.class, String[].class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, new String[] {routerFile}});
+  }
+
+  @Test
+  public void testProxyAppend() throws Exception {
+
+    // Append a test string via router
+    EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.APPEND);
+    DFSClient routerClient = getRouterContext().getClient();
+    HdfsDataOutputStream stream =
+        routerClient.append(routerFile, 1024, createFlag, null, null);
+    stream.writeBytes(TEST_STRING);
+    stream.close();
+
+    // Verify file size via NN
+    FileStatus status = getFileStatus(nnFS, nnFile);
+    assertTrue(status.getLen() > TEST_STRING.length());
+
+    // Validate router failure response matches NN failure response.
+    Method m = ClientProtocol.class.getMethod("append", String.class,
+        String.class, EnumSetWritable.class);
+    String badPath = "/unknownlocation/unknowndir";
+    EnumSetWritable<CreateFlag> createFlagWritable =
+        new EnumSetWritable<CreateFlag>(createFlag);
+    compareResponses(routerProtocol, nnProtocol, m,
+        new Object[] {badPath, "testClient", createFlagWritable});
+  }
+
+  @Test
+  public void testProxyGetAdditionalDatanode()
+      throws IOException, InterruptedException, URISyntaxException {
+
+    // Use primitive APIs to open a file, add a block, and get datanode location
+    EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.CREATE);
+    String clientName = getRouterContext().getClient().getClientName();
+    String newRouterFile = routerFile + "_additionalDatanode";
+    HdfsFileStatus status = routerProtocol.create(
+        newRouterFile, new FsPermission("777"), clientName,
+        new EnumSetWritable<CreateFlag>(createFlag), true, (short) 1,
+        (long) 1024, CryptoProtocolVersion.supported(), null);
+
+    // Add a block via router (requires client to have same lease)
+    LocatedBlock block = routerProtocol.addBlock(
+        newRouterFile, clientName, null, null,
+        status.getFileId(), null, null);
+
+    DatanodeInfo[] exclusions = new DatanodeInfo[0];
+    LocatedBlock newBlock = routerProtocol.getAdditionalDatanode(
+        newRouterFile, status.getFileId(), block.getBlock(),
+        block.getLocations(), block.getStorageIDs(), exclusions, 1, clientName);
+    assertNotNull(newBlock);
+  }
+
+  @Test
+  public void testProxyCreateFileAlternateUser()
+      throws IOException, URISyntaxException, InterruptedException {
+
+    // Create via Router
+    String routerDir = cluster.getFederatedTestDirectoryForNS(ns);
+    String namenodeDir = cluster.getNamenodeTestDirectoryForNS(ns);
+    String newRouterFile = routerDir + "/unknownuser";
+    String newNamenodeFile = namenodeDir + "/unknownuser";
+    String username = "unknownuser";
+
+    // Allow all user access to dir
+    namenode.getFileContext().setPermission(
+        new Path(namenodeDir), new FsPermission("777"));
+
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(username);
+    DFSClient client = getRouterContext().getClient(ugi);
+    client.create(newRouterFile, true);
+
+    // Fetch via NN and check user
+    FileStatus status = getFileStatus(nnFS, newNamenodeFile);
+    assertEquals(status.getOwner(), username);
+  }
+
+  @Test
+  public void testProxyGetFileInfoAcessException() throws IOException {
+
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser("unknownuser");
+
+    // List files from the NN and trap the exception
+    Exception nnFailure = null;
+    try {
+      String testFile = cluster.getNamenodeTestFileForNS(ns);
+      namenode.getClient(ugi).getLocatedBlocks(testFile, 0);
+    } catch (Exception e) {
+      nnFailure = e;
+    }
+    assertNotNull(nnFailure);
+
+    // List files from the router and trap the exception
+    Exception routerFailure = null;
+    try {
+      String testFile = cluster.getFederatedTestFileForNS(ns);
+      getRouterContext().getClient(ugi).getLocatedBlocks(testFile, 0);
+    } catch (Exception e) {
+      routerFailure = e;
+    }
+    assertNotNull(routerFailure);
+
+    assertEquals(routerFailure.getClass(), nnFailure.getClass());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a60031c7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
new file mode 100644
index 0000000..5489691
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java
@@ -0,0 +1,216 @@
+/**
+ * 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 static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.createFile;
+import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.federation.MockResolver;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.NamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+
+/**
+ * The the RPC interface of the {@link getRouter()} implemented by
+ * {@link RouterRpcServer}.
+ */
+public class TestRouterRpcMultiDestination extends TestRouterRpc {
+
+  @Override
+  public void testSetup() throws Exception {
+
+    RouterDFSCluster cluster = getCluster();
+
+    // Create mock locations
+    getCluster().installMockLocations();
+    List<RouterContext> routers = cluster.getRouters();
+
+    // Add extra location to the root mount / such that the root mount points:
+    // /
+    //   ns0 -> /
+    //   ns1 -> /
+    for (RouterContext rc : routers) {
+      Router router = rc.getRouter();
+      MockResolver resolver = (MockResolver) router.getSubclusterResolver();
+      resolver.addLocation("/", cluster.getNameservices().get(1), "/");
+    }
+
+    // Create a mount that points to 2 dirs in the same ns:
+    // /same
+    //   ns0 -> /
+    //   ns0 -> /target-ns0
+    for (RouterContext rc : routers) {
+      Router router = rc.getRouter();
+      MockResolver resolver = (MockResolver) router.getSubclusterResolver();
+      List<String> nss = cluster.getNameservices();
+      String ns0 = nss.get(0);
+      resolver.addLocation("/same", ns0, "/");
+      resolver.addLocation("/same", ns0, cluster.getNamenodePathForNS(ns0));
+    }
+
+    // Delete all files via the NNs and verify
+    cluster.deleteAllFiles();
+
+    // Create test fixtures on NN
+    cluster.createTestDirectoriesNamenode();
+
+    // Wait to ensure NN has fully created its test directories
+    Thread.sleep(100);
+
+    // Pick a NS, namenode and getRouter() for this test
+    RouterContext router = cluster.getRandomRouter();
+    this.setRouter(router);
+
+    String ns = cluster.getRandomNameservice();
+    this.setNs(ns);
+    this.setNamenode(cluster.getNamenode(ns, null));
+
+    // Create a test file on a single NN that is accessed via a getRouter() path
+    // with 2 destinations. All tests should failover to the alternate
+    // destination if the wrong NN is attempted first.
+    Random r = new Random();
+    String randomString = "testfile-" + r.nextInt();
+    setNamenodeFile("/" + randomString);
+    setRouterFile("/" + randomString);
+
+    FileSystem nnFs = getNamenodeFileSystem();
+    FileSystem routerFs = getRouterFileSystem();
+    createFile(nnFs, getNamenodeFile(), 32);
+
+    verifyFileExists(nnFs, getNamenodeFile());
+    verifyFileExists(routerFs, getRouterFile());
+  }
+
+  private void testListing(String path) throws IOException {
+
+    // Collect the mount table entries for this path
+    Set<String> requiredPaths = new TreeSet<>();
+    RouterContext rc = getRouterContext();
+    Router router = rc.getRouter();
+    FileSubclusterResolver subclusterResolver = router.getSubclusterResolver();
+    for (String mount : subclusterResolver.getMountPoints(path)) {
+      requiredPaths.add(mount);
+    }
+
+    // Get files/dirs from the Namenodes
+    PathLocation location = subclusterResolver.getDestinationForPath(path);
+    for (RemoteLocation loc : location.getDestinations()) {
+      String nsId = loc.getNameserviceId();
+      String dest = loc.getDest();
+      NamenodeContext nn = getCluster().getNamenode(nsId, null);
+      FileSystem fs = nn.getFileSystem();
+      FileStatus[] files = fs.listStatus(new Path(dest));
+      for (FileStatus file : files) {
+        String pathName = file.getPath().getName();
+        requiredPaths.add(pathName);
+      }
+    }
+
+    // Get files/dirs from the Router
+    DirectoryListing listing =
+        getRouterProtocol().getListing(path, HdfsFileStatus.EMPTY_NAME, false);
+    Iterator<String> requiredPathsIterator = requiredPaths.iterator();
+
+    // Match each path returned and verify order returned
+    HdfsFileStatus[] partialListing = listing.getPartialListing();
+    for (HdfsFileStatus fileStatus : listing.getPartialListing()) {
+      String fileName = requiredPathsIterator.next();
+      String currentFile = fileStatus.getFullPath(new Path(path)).getName();
+      assertEquals(currentFile, fileName);
+    }
+
+    // Verify the total number of results found/matched
+    assertEquals(
+        requiredPaths + " doesn't match " + Arrays.toString(partialListing),
+        requiredPaths.size(), partialListing.length);
+  }
+
+  @Override
+  public void testProxyListFiles() throws IOException, InterruptedException,
+      URISyntaxException, NoSuchMethodException, SecurityException {
+
+    // Verify that the root listing is a union of the mount table destinations
+    // and the files stored at all nameservices mounted at the root (ns0 + ns1)
+    // / -->
+    // /ns0 (from mount table)
+    // /ns1 (from mount table)
+    // /same (from the mount table)
+    // all items in / of ns0 from mapping of / -> ns0:::/)
+    // all items in / of ns1 from mapping of / -> ns1:::/)
+    testListing("/");
+
+    // Verify that the "/same" mount point lists the contents of both dirs in
+    // the same ns
+    // /same -->
+    // /target-ns0 (from root of ns0)
+    // /testdir (from contents of /target-ns0)
+    testListing("/same");
+
+    // List a non-existing path and validate error response with NN behavior
+    ClientProtocol namenodeProtocol =
+        getCluster().getRandomNamenode().getClient().getNamenode();
+    Method m = ClientProtocol.class.getMethod(
+        "getListing", String.class,  byte[].class, boolean.class);
+    String badPath = "/unknownlocation/unknowndir";
+    compareResponses(getRouterProtocol(), namenodeProtocol, m,
+        new Object[] {badPath, HdfsFileStatus.EMPTY_NAME, false});
+  }
+
+  @Override
+  public void testProxyRenameFiles() throws IOException, InterruptedException {
+
+    super.testProxyRenameFiles();
+
+    List<String> nss = getCluster().getNameservices();
+    String ns0 = nss.get(0);
+    String ns1 = nss.get(1);
+
+    // Rename a file from ns0 into the root (mapped to both ns0 and ns1)
+    String testDir0 = getCluster().getFederatedTestDirectoryForNS(ns0);
+    String filename0 = testDir0 + "/testrename";
+    String renamedFile = "/testrename";
+    testRename(getRouterContext(), filename0, renamedFile, false);
+    testRename2(getRouterContext(), filename0, renamedFile, false);
+
+    // Rename a file from ns1 into the root (mapped to both ns0 and ns1)
+    String testDir1 = getCluster().getFederatedTestDirectoryForNS(ns1);
+    String filename1 = testDir1 + "/testrename";
+    testRename(getRouterContext(), filename1, renamedFile, false);
+    testRename2(getRouterContext(), filename1, renamedFile, false);
+  }
+}
\ 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


[40/50] [abbrv] hadoop git commit: HDFS-12062. removeErasureCodingPolicy needs super user permission. Contributed by Wei-Chiu Chuang.

Posted by in...@apache.org.
HDFS-12062. removeErasureCodingPolicy needs super user permission. Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/HDFS-10467
Commit: 369f731264d77617452e4074d15404bd62ec6093
Parents: 9902be7
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Jul 28 00:50:08 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Jul 28 00:51:03 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 26 ++++++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  1 +
 .../hadoop/hdfs/TestDistributedFileSystem.java  | 50 ++++++++++++++++++++
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 20 ++++++++
 4 files changed, 93 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/369f7312/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 8acda61..677ea35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2774,25 +2774,43 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public AddECPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkOpen();
-    return namenode.addErasureCodingPolicies(policies);
+    try (TraceScope ignored = tracer.newScope("addErasureCodingPolicies")) {
+      return namenode.addErasureCodingPolicies(policies);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
   }
 
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.removeErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("removeErasureCodingPolicy")) {
+      namenode.removeErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
   }
 
   public void enableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.enableErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("enableErasureCodingPolicy")) {
+      namenode.enableErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
+    }
   }
 
   public void disableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.disableErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("disableErasureCodingPolicy")) {
+      namenode.disableErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
+    }
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/369f7312/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 39d93df..9cd58cb 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
@@ -2304,6 +2304,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
     namesystem.removeErasureCodingPolicy(ecPolicyName);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/369f7312/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 b35d374..9525609 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
@@ -93,6 +93,7 @@ import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
@@ -1561,6 +1562,27 @@ public class TestDistributedFileSystem {
       fs.removeErasureCodingPolicy(policyName);
       assertEquals(policyName, ErasureCodingPolicyManager.getInstance().
           getRemovedPolicies().get(0).getName());
+
+      // remove erasure coding policy as a user without privilege
+      UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+          "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+      final MiniDFSCluster finalCluster = cluster;
+      fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          DistributedFileSystem fs = finalCluster.getFileSystem();
+          try {
+            fs.removeErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          return null;
+        }
+      });
+
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -1609,6 +1631,34 @@ public class TestDistributedFileSystem {
         GenericTestUtils.assertExceptionContains("does not exists", e);
         // pass
       }
+
+      // disable and enable erasure coding policy as a user without privilege
+      UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+          "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+      final MiniDFSCluster finalCluster = cluster;
+      fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          DistributedFileSystem fs = finalCluster.getFileSystem();
+          try {
+            fs.disableErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                    "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          try {
+            fs.enableErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                    "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          return null;
+        }
+      });
     } finally {
       if (cluster != null) {
         cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/369f7312/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index f90a2f3..127dad1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -693,5 +693,25 @@ public class TestErasureCodingPolicies {
     assertTrue(responses[0].isSucceed());
     assertEquals(SystemErasureCodingPolicies.getPolicies().size() + 1,
         ErasureCodingPolicyManager.getInstance().getPolicies().length);
+
+    // add erasure coding policy as a user without privilege
+    UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+        "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+    final ErasureCodingPolicy ecPolicy = newPolicy;
+    fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        DistributedFileSystem fs = cluster.getFileSystem();
+        try {
+          fs.addErasureCodingPolicies(new ErasureCodingPolicy[]{ecPolicy});
+          fail();
+        } catch (AccessControlException ace) {
+          GenericTestUtils.assertExceptionContains("Access denied for user " +
+                  "ProbablyNotARealUserName. Superuser privilege is required",
+              ace);
+        }
+        return null;
+      }
+    });
   }
 }


---------------------------------------------------------------------
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: Revert "YARN-6804. [YARN core changes] Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi"

Posted by in...@apache.org.
Revert "YARN-6804. [YARN core changes] Allow custom hostname for docker containers in native services. Contributed by Billie Rinaldi"

This reverts commit 4a771d9010de0867ac901bead075383ddf1f30dc.


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

Branch: refs/heads/HDFS-10467
Commit: 2054324d471a6152ba34284b4259d4fd09c71704
Parents: 465c213
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Sun Jul 23 12:56:18 2017 +0800
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Sun Jul 23 12:56:18 2017 +0800

----------------------------------------------------------------------
 .../client/binding/RegistryPathUtils.java       |  2 +-
 .../hadoop/registry/client/types/Endpoint.java  |  4 +-
 .../registry/client/types/ServiceRecord.java    |  4 +-
 .../hadoop-yarn-server-nodemanager/pom.xml      |  4 --
 .../runtime/DockerLinuxContainerRuntime.java    | 67 +++++---------------
 .../linux/runtime/docker/DockerRunCommand.java  |  6 --
 .../impl/container-executor.c                   |  4 --
 .../test/test-container-executor.c              | 16 ++---
 .../runtime/TestDockerContainerRuntime.java     | 58 ++++-------------
 9 files changed, 44 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
index 5fa45f9..5d8ea3f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java
@@ -213,6 +213,6 @@ public class RegistryPathUtils {
    * @return a string suitable for use in registry paths.
    */
   public static String encodeYarnID(String yarnId) {
-    return yarnId.replace("container", "ctr").replace("_", "-");
+    return yarnId.replace("_", "-");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
index 392884f..395f836 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/Endpoint.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -46,7 +46,7 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @JsonIgnoreProperties(ignoreUnknown = true)
-@JsonInclude(JsonInclude.Include.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public final class Endpoint implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
index d40866a..674d6d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/types/ServiceRecord.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.registry.client.types;
 
 import com.fasterxml.jackson.annotation.JsonAnyGetter;
 import com.fasterxml.jackson.annotation.JsonAnySetter;
-import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -37,7 +37,7 @@ import java.util.Map;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-@JsonInclude(JsonInclude.Include.NON_NULL)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public class ServiceRecord implements Cloneable {
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index 094519a..a0f4ef7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -52,10 +52,6 @@
       <artifactId>hadoop-yarn-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-registry</artifactId>
-    </dependency>
-    <dependency>
       <groupId>javax.xml.bind</groupId>
       <artifactId>jaxb-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.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/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index e058d6e..8db03bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
@@ -102,11 +101,6 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  *     property.
  *   </li>
  *   <li>
- *     {@code YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME} sets the
- *     hostname to be used by the Docker container. If not specified, a
- *     hostname will be derived from the container ID.
- *   </li>
- *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER}
  *     controls whether the Docker container is a privileged container. In order
  *     to use privileged containers, the
@@ -140,10 +134,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       "^(([a-zA-Z0-9.-]+)(:\\d+)?/)?([a-z0-9_./-]+)(:[\\w.-]+)?$";
   private static final Pattern dockerImagePattern =
       Pattern.compile(DOCKER_IMAGE_PATTERN);
-  public static final String HOSTNAME_PATTERN =
-      "^[a-zA-Z0-9][a-zA-Z0-9_.-]+$";
-  private static final Pattern hostnamePattern = Pattern.compile(
-      HOSTNAME_PATTERN);
 
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_IMAGE =
@@ -157,10 +147,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_NETWORK =
       "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
-  @InterfaceAudience.Private
-  public static final String ENV_DOCKER_CONTAINER_HOSTNAME =
-      "YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME";
-  @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER =
       "YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER";
   @InterfaceAudience.Private
@@ -225,7 +211,9 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     this.privilegedOperationExecutor = privilegedOperationExecutor;
 
     if (cGroupsHandler == null) {
-      LOG.info("cGroupsHandler is null - cgroups not in use.");
+      if (LOG.isInfoEnabled()) {
+        LOG.info("cGroupsHandler is null - cgroups not in use.");
+      }
     } else {
       this.cGroupsHandler = cGroupsHandler;
     }
@@ -279,29 +267,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     throw new ContainerExecutionException(msg);
   }
 
-  public static void validateHostname(String hostname) throws
-      ContainerExecutionException {
-    if (hostname != null && !hostname.isEmpty()) {
-      if (!hostnamePattern.matcher(hostname).matches()) {
-        throw new ContainerExecutionException("Hostname '" + hostname
-            + "' doesn't match docker hostname pattern");
-      }
-    }
-  }
-
-  /** Set a DNS friendly hostname. */
-  private void setHostname(DockerRunCommand runCommand, String
-      containerIdStr, String name)
-      throws ContainerExecutionException {
-    if (name == null || name.isEmpty()) {
-      name = RegistryPathUtils.encodeYarnID(containerIdStr);
-      validateHostname(name);
-    }
-
-    LOG.info("setting hostname in container to: " + name);
-    runCommand.setHostname(name);
-  }
-
   /**
    * If CGROUPS in enabled and not set to none, then set the CGROUP parent for
    * the command instance.
@@ -378,8 +343,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       return false;
     }
 
-    LOG.info("Privileged container requested for : " + container
-        .getContainerId().toString());
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Privileged container requested for : " + container
+          .getContainerId().toString());
+    }
 
     //Ok, so we have been asked to run a privileged container. Security
     // checks need to be run. Each violation is an error.
@@ -408,8 +375,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       throw new ContainerExecutionException(message);
     }
 
-    LOG.info("All checks pass. Launching privileged container for : "
-        + container.getContainerId().toString());
+    if (LOG.isInfoEnabled()) {
+      LOG.info("All checks pass. Launching privileged container for : "
+          + container.getContainerId().toString());
+    }
 
     return true;
   }
@@ -444,7 +413,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         .getEnvironment();
     String imageName = environment.get(ENV_DOCKER_CONTAINER_IMAGE);
     String network = environment.get(ENV_DOCKER_CONTAINER_NETWORK);
-    String hostname = environment.get(ENV_DOCKER_CONTAINER_HOSTNAME);
 
     if(network == null || network.isEmpty()) {
       network = defaultNetwork;
@@ -452,8 +420,6 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
 
     validateContainerNetworkType(network);
 
-    validateHostname(hostname);
-
     validateImageName(imageName);
 
     String containerIdStr = container.getContainerId().toString();
@@ -484,13 +450,12 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         runAsUser, imageName)
         .detachOnRun()
         .setContainerWorkDir(containerWorkDir.toString())
-        .setNetworkType(network);
-    setHostname(runCommand, containerIdStr, hostname);
-    runCommand.setCapabilities(capabilities)
+        .setNetworkType(network)
+        .setCapabilities(capabilities)
         .addMountLocation(CGROUPS_ROOT_DIRECTORY,
             CGROUPS_ROOT_DIRECTORY + ":ro", false);
-
     List<String> allDirs = new ArrayList<>(containerLocalDirs);
+
     allDirs.addAll(filecacheDirs);
     allDirs.add(containerWorkDir.toString());
     allDirs.addAll(containerLogDirs);
@@ -528,7 +493,9 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
         ENV_DOCKER_CONTAINER_RUN_OVERRIDE_DISABLE);
 
     if (disableOverride != null && disableOverride.equals("true")) {
-      LOG.info("command override disabled");
+      if (LOG.isInfoEnabled()) {
+        LOG.info("command override disabled");
+      }
     } else {
       List<String> overrideCommands = new ArrayList<>();
       Path launchDst =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.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/linux/runtime/docker/DockerRunCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
index b645754..f79f4ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
@@ -91,12 +91,6 @@ public class DockerRunCommand extends DockerCommand {
 
     return this;
   }
-
-  public DockerRunCommand setHostname(String hostname) {
-    super.addCommandArguments("--hostname=" + hostname);
-    return this;
-  }
-
   public DockerRunCommand addDevice(String sourceDevice, String
       destinationDevice) {
     super.addCommandArguments("--device=" + sourceDevice + ":" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index 5070d62..5d138f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -1215,7 +1215,6 @@ char* sanitize_docker_command(const char *line) {
     {"rm", no_argument, 0, 'r' },
     {"workdir", required_argument, 0, 'w' },
     {"net", required_argument, 0, 'e' },
-    {"hostname", required_argument, 0, 'h' },
     {"cgroup-parent", required_argument, 0, 'g' },
     {"privileged", no_argument, 0, 'p' },
     {"cap-add", required_argument, 0, 'a' },
@@ -1257,9 +1256,6 @@ char* sanitize_docker_command(const char *line) {
       case 'e':
         quote_and_append_arg(&output, &output_size, "--net=", optarg);
         break;
-      case 'h':
-        quote_and_append_arg(&output, &output_size, "--hostname=", optarg);
-        break;
       case 'v':
         quote_and_append_arg(&output, &output_size, "-v ", optarg);
         break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index b7d0e44..83d11ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -1088,17 +1088,17 @@ void test_trim_function() {
 void test_sanitize_docker_command() {
 
   char *input[] = {
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
-    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --hostname=test.host.name --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=$CID --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
+    "run --name=cname --user=nobody -d --workdir=/yarn/local/cdir --privileged --rm --device=/sys/fs/cgroup/device:/sys/fs/cgroup/device --detach=true --cgroup-parent=/sys/fs/cgroup/cpu/yarn/cid --net=host --cap-drop=ALL --cap-add=SYS_CHROOT --cap-add=MKNOD --cap-add=SETFCAP --cap-add=SETPCAP --cap-add=FSETID --cap-add=CHOWN --cap-add=AUDIT_WRITE --cap-add=SETGID --cap-add=NET_RAW --cap-add=FOWNER --cap-add=SETUID --cap-add=DAC_OVERRIDE --cap-add=KILL --cap-add=NET_BIND_SERVICE -v /sys/fs/cgroup:/sys/fs/cgroup:ro -v /yarn/local/cdir:/yarn/local/cdir -v /yarn/local/usercache/test/:/yarn/local/usercache/test/ ubuntu' || touch /tmp/file # bash /yarn/local/usercache/test/appcache/aid/cid/launch_container.sh",
     "run ''''''''"
   };
   char *expected_output[] = {
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
-      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --hostname='test.host.name' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='$CID' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
+      "run --name='cname' --user='nobody' -d --workdir='/yarn/local/cdir' --privileged --rm --device='/sys/fs/cgroup/device:/sys/fs/cgroup/device' --detach='true' --cgroup-parent='/sys/fs/cgroup/cpu/yarn/cid' --net='host' --cap-drop='ALL' --cap-add='SYS_CHROOT' --cap-add='MKNOD' --cap-add='SETFCAP' --cap-add='SETPCAP' --cap-add='FSETID' --cap-add='CHOWN' --cap-add='AUDIT_WRITE' --cap-add='SETGID' --cap-add='NET_RAW' --cap-add='FOWNER' --cap-add='SETUID' --cap-add='DAC_OVERRIDE' --cap-add='KILL' --cap-add='NET_BIND_SERVICE' -v '/sys/fs/cgroup:/sys/fs/cgroup:ro' -v '/yarn/local/cdir:/yarn/local/cdir' -v '/yarn/local/usercache/test/:/yarn/local/usercache/test/' 'ubuntu'\"'\"'' '||' 'touch' '/tmp/file' '#' 'bash' '/yarn/local/usercache/test/appcache/aid/cid/launch_container.sh' ",
       "run ''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"''\"'\"'' ",
   };
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2054324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.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/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index 9894dcd..f611843 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -70,7 +69,6 @@ public class TestDockerContainerRuntime {
   private PrivilegedOperationExecutor mockExecutor;
   private CGroupsHandler mockCGroupsHandler;
   private String containerId;
-  private String defaultHostname;
   private Container container;
   private ContainerId cId;
   private ContainerLaunchContext context;
@@ -110,7 +108,6 @@ public class TestDockerContainerRuntime {
         .mock(PrivilegedOperationExecutor.class);
     mockCGroupsHandler = Mockito.mock(CGroupsHandler.class);
     containerId = "container_id";
-    defaultHostname = RegistryPathUtils.encodeYarnID(containerId);
     container = mock(Container.class);
     cId = mock(ContainerId.class);
     context = mock(ContainerLaunchContext.class);
@@ -290,7 +287,6 @@ public class TestDockerContainerRuntime {
         .append("--user=%2$s -d ")
         .append("--workdir=%3$s ")
         .append("--net=host ")
-        .append("--hostname=" + defaultHostname + " ")
         .append(getExpectedTestCapabilitiesArgumentString())
         .append(getExpectedCGroupsMountString())
         .append("-v %4$s:%4$s ")
@@ -369,7 +365,7 @@ public class TestDockerContainerRuntime {
     String disallowedNetwork = "sdn" + Integer.toString(randEngine.nextInt());
 
     try {
-      env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
           disallowedNetwork);
       runtime.launchContainer(builder.build());
       Assert.fail("Network was expected to be disallowed: " +
@@ -382,11 +378,8 @@ public class TestDockerContainerRuntime {
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS.length;
     String allowedNetwork = YarnConfiguration
         .DEFAULT_NM_DOCKER_ALLOWED_CONTAINER_NETWORKS[randEngine.nextInt(size)];
-    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
         allowedNetwork);
-    String expectedHostname = "test.hostname";
-    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_HOSTNAME,
-        expectedHostname);
 
     //this should cause no failures.
 
@@ -400,7 +393,6 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + allowedNetwork + " ")
-            .append("--hostname=" + expectedHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -456,7 +448,6 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork1 + " ")
-            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -480,7 +471,7 @@ public class TestDockerContainerRuntime {
     //now set an explicit (non-default) allowedNetwork and ensure that it is
     // used.
 
-    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
         customNetwork2);
     runtime.launchContainer(builder.build());
 
@@ -494,7 +485,6 @@ public class TestDockerContainerRuntime {
         new StringBuffer("run --name=%1$s ").append("--user=%2$s -d ")
             .append("--workdir=%3$s ")
             .append("--net=" + customNetwork2 + " ")
-            .append("--hostname=" + defaultHostname + " ")
             .append(getExpectedTestCapabilitiesArgumentString())
             .append(getExpectedCGroupsMountString())
             .append("-v %4$s:%4$s ").append("-v %5$s:%5$s ")
@@ -515,7 +505,7 @@ public class TestDockerContainerRuntime {
 
     //disallowed network should trigger a launch failure
 
-    env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_NETWORK,
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK",
         customNetwork3);
     try {
       runtime.launchContainer(builder.build());
@@ -534,8 +524,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "invalid-value");
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "invalid-value");
     runtime.launchContainer(builder.build());
 
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -562,8 +552,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -585,8 +575,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
     //By default
     // yarn.nodemanager.runtime.linux.docker.privileged-containers.acl
     // is empty. So we expect this launch to fail.
@@ -615,8 +605,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
 
     try {
       runtime.launchContainer(builder.build());
@@ -642,8 +632,8 @@ public class TestDockerContainerRuntime {
         mockExecutor, mockCGroupsHandler);
     runtime.initialize(conf);
 
-    env.put(DockerLinuxContainerRuntime
-            .ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER, "true");
+    env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER",
+        "true");
 
     runtime.launchContainer(builder.build());
     PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
@@ -937,24 +927,4 @@ public class TestDockerContainerRuntime {
       }
     }
   }
-
-  @Test
-  public void testDockerHostnamePattern() throws Exception {
-    String[] validNames = {"ab", "a.b.c.d", "a1-b.cd.ef", "0AB.", "C_D-"};
-
-    String[] invalidNames = {"a", "a#.b.c", "-a.b.c", "a@b.c", "a/b/c"};
-
-    for (String name : validNames) {
-      DockerLinuxContainerRuntime.validateHostname(name);
-    }
-
-    for (String name : invalidNames) {
-      try {
-        DockerLinuxContainerRuntime.validateHostname(name);
-        Assert.fail(name + " is an invalid hostname and should fail the regex");
-      } catch (ContainerExecutionException ce) {
-        continue;
-      }
-    }
-  }
 }


---------------------------------------------------------------------
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: HADOOP-14692. Upgrade Apache Rat

Posted by in...@apache.org.
HADOOP-14692. Upgrade Apache Rat

Signed-off-by: Anu Engineer <ae...@apache.org>


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

Branch: refs/heads/HDFS-10467
Commit: 5f4808ce73a373e646ce324b0037dca54e8adc1e
Parents: c4a85c6
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Jul 27 13:04:32 2017 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Thu Jul 27 13:04:50 2017 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f4808ce/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e7f6eeb..29524a4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -102,7 +102,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
     <maven-gpg-plugin.version>1.5</maven-gpg-plugin.version>
     <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <maven-resources-plugin.version>3.0.1</maven-resources-plugin.version>
-    <apache-rat-plugin.version>0.10</apache-rat-plugin.version>
+    <apache-rat-plugin.version>0.12</apache-rat-plugin.version>
     <wagon-ssh.version>1.0</wagon-ssh.version>
     <maven-clover2-plugin.version>3.3.0</maven-clover2-plugin.version>
     <maven-bundle-plugin.version>2.5.0</maven-bundle-plugin.version>


---------------------------------------------------------------------
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-12190. Enable 'hdfs dfs -stat' to display access time. Contributed by Yongjun Zhang.

Posted by in...@apache.org.
HDFS-12190. Enable 'hdfs dfs -stat' to display access time. Contributed by Yongjun Zhang.


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

Branch: refs/heads/HDFS-10467
Commit: c6330f22a5e5c2370bab885f9bea4bf8f5e9cf44
Parents: e3c7300
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Thu Jul 27 16:48:24 2017 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Thu Jul 27 16:48:24 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/shell/Stat.java    | 19 ++++++++++++++-----
 .../src/site/markdown/FileSystemShell.md         |  4 ++--
 .../src/test/resources/testConf.xml              | 10 +++++++---
 .../org/apache/hadoop/hdfs/TestDFSShell.java     | 12 ++++++++++--
 4 files changed, 33 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6330f22/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
index cf8270e..8c624cc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
@@ -40,8 +40,10 @@ import org.apache.hadoop.fs.FileStatus;
  *   %o: Block size<br>
  *   %r: replication<br>
  *   %u: User name of owner<br>
- *   %y: UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;<br>
- *   %Y: Milliseconds since January 1, 1970 UTC<br>
+ *   %x: atime UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;<br>
+ *   %X: atime Milliseconds since January 1, 1970 UTC<br>
+ *   %y: mtime UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;<br>
+ *   %Y: mtime Milliseconds since January 1, 1970 UTC<br>
  * If the format is not specified, %y is used by default.
  */
 @InterfaceAudience.Private
@@ -62,9 +64,10 @@ class Stat extends FsCommand {
     "octal (%a) and symbolic (%A), filesize in" + NEWLINE +
     "bytes (%b), type (%F), group name of owner (%g)," + NEWLINE +
     "name (%n), block size (%o), replication (%r), user name" + NEWLINE +
-    "of owner (%u), modification date (%y, %Y)." + NEWLINE +
-    "%y shows UTC date as \"yyyy-MM-dd HH:mm:ss\" and" + NEWLINE +
-    "%Y shows milliseconds since January 1, 1970 UTC." + NEWLINE +
+    "of owner (%u), access date (%x, %X)." + NEWLINE +
+    "modification date (%y, %Y)." + NEWLINE +
+    "%x and %y show UTC date as \"yyyy-MM-dd HH:mm:ss\" and" + NEWLINE +
+    "%X and %Y show milliseconds since January 1, 1970 UTC." + NEWLINE +
     "If the format is not specified, %y is used by default." + NEWLINE;
 
   protected final SimpleDateFormat timeFmt;
@@ -127,6 +130,12 @@ class Stat extends FsCommand {
           case 'u':
             buf.append(stat.getOwner());
             break;
+          case 'x':
+            buf.append(timeFmt.format(new Date(stat.getAccessTime())));
+            break;
+          case 'X':
+            buf.append(stat.getAccessTime());
+            break;
           case 'y':
             buf.append(timeFmt.format(new Date(stat.getModificationTime())));
             break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6330f22/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 0a594ab..71eec75 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -676,11 +676,11 @@ stat
 
 Usage: `hadoop fs -stat [format] <path> ...`
 
-Print statistics about the file/directory at \<path\> in the specified format. Format accepts permissions in octal (%a) and symbolic (%A), filesize in bytes (%b), type (%F), group name of owner (%g), name (%n), block size (%o), replication (%r), user name of owner(%u), and modification date (%y, %Y). %y shows UTC date as "yyyy-MM-dd HH:mm:ss" and %Y shows milliseconds since January 1, 1970 UTC. If the format is not specified, %y is used by default.
+Print statistics about the file/directory at \<path\> in the specified format. Format accepts permissions in octal (%a) and symbolic (%A), filesize in bytes (%b), type (%F), group name of owner (%g), name (%n), block size (%o), replication (%r), user name of owner(%u), access date(%x, %X), and modification date (%y, %Y). %x and %y show UTC date as "yyyy-MM-dd HH:mm:ss", and %X and %Y show milliseconds since January 1, 1970 UTC. If the format is not specified, %y is used by default.
 
 Example:
 
-* `hadoop fs -stat "%F %a %u:%g %b %y %n" /file`
+* `hadoop fs -stat "type:%F perm:%a %u:%g size:%b mtime:%y atime:%x name:%n" /file`
 
 Exit Code: Returns 0 on success and -1 on error.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6330f22/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 64677f8..6a3d53a 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -919,15 +919,19 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*of owner \(%u\), modification date \(%y, %Y\).( )*</expected-output>
+          <expected-output>^( |\t)*of owner \(%u\), access date \(%x, %X\).( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*%y shows UTC date as "yyyy-MM-dd HH:mm:ss" and( )*</expected-output>
+          <expected-output>^( |\t)*modification date \(%y, %Y\).( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*%Y shows milliseconds since January 1, 1970 UTC.( )*</expected-output>
+          <expected-output>^( |\t)*%x and %y show UTC date as "yyyy-MM-dd HH:mm:ss" and( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*%X and %Y show milliseconds since January 1, 1970 UTC.( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6330f22/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index c82c045..27d41b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -36,12 +36,12 @@ import java.util.zip.GZIPOutputStream;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
+
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Level;
 import org.junit.Test;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -65,6 +65,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.rules.Timeout;
 import org.junit.AfterClass;
@@ -115,6 +116,7 @@ public class TestDFSShell {
         GenericTestUtils.getTestDir("TestDFSShell").getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1000);
 
     miniCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     miniCluster.waitActive();
@@ -2002,8 +2004,12 @@ public class TestDFSShell {
     DFSTestUtil.createFile(dfs, testFile2, 2 * BLOCK_SIZE, (short) 3, 0);
     final FileStatus status1 = dfs.getFileStatus(testDir1);
     final String mtime1 = fmt.format(new Date(status1.getModificationTime()));
+    final String atime1 = fmt.format(new Date(status1.getAccessTime()));
+    long now = Time.now();
+    dfs.setTimes(testFile2, now + 3000, now + 6000);
     final FileStatus status2 = dfs.getFileStatus(testFile2);
     final String mtime2 = fmt.format(new Date(status2.getModificationTime()));
+    final String atime2 = fmt.format(new Date(status2.getAccessTime()));
 
     final ByteArrayOutputStream out = new ByteArrayOutputStream();
     System.setOut(new PrintStream(out));
@@ -2036,17 +2042,19 @@ public class TestDFSShell {
         out.toString().contains(String.valueOf(octal)));
 
     out.reset();
-    doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %y %n", testDir1, testFile2);
+    doFsStat(dfs.getConf(), "%F %a %A %u:%g %b %x %y %n", testDir1, testFile2);
 
     n = status2.getPermission().toShort();
     octal = (n>>>9&1)*1000 + (n>>>6&7)*100 + (n>>>3&7)*10 + (n&7);
     assertTrue(out.toString(), out.toString().contains(mtime1));
+    assertTrue(out.toString(), out.toString().contains(atime1));
     assertTrue(out.toString(), out.toString().contains("regular file"));
     assertTrue(out.toString(),
         out.toString().contains(status2.getPermission().toString()));
     assertTrue(out.toString(),
         out.toString().contains(String.valueOf(octal)));
     assertTrue(out.toString(), out.toString().contains(mtime2));
+    assertTrue(out.toString(), out.toString().contains(atime2));
   }
 
   private static void doFsStat(Configuration conf, String format, Path... files)


---------------------------------------------------------------------
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-14597. Native compilation broken with OpenSSL-1.1.0. Contributed by Ravi Prakash.

Posted by in...@apache.org.
HADOOP-14597. Native compilation broken with OpenSSL-1.1.0. Contributed by Ravi Prakash.


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

Branch: refs/heads/HDFS-10467
Commit: 94ca52ae9ec0ae04854d726bf2ac1bc457b96a9c
Parents: 1058362
Author: Ravi Prakash <ra...@apache.org>
Authored: Mon Jul 24 16:01:45 2017 -0700
Committer: Ravi Prakash <ra...@apache.org>
Committed: Mon Jul 24 16:01:45 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/crypto/OpensslCipher.c    | 46 ++++++++++++++++++--
 .../src/main/native/pipes/impl/HadoopPipes.cc   | 12 ++++-
 2 files changed, 53 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94ca52ae/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c
index 5cb5bba..c7984a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c
@@ -30,6 +30,11 @@ static void (*dlsym_EVP_CIPHER_CTX_free)(EVP_CIPHER_CTX *);
 static int (*dlsym_EVP_CIPHER_CTX_cleanup)(EVP_CIPHER_CTX *);
 static void (*dlsym_EVP_CIPHER_CTX_init)(EVP_CIPHER_CTX *);
 static int (*dlsym_EVP_CIPHER_CTX_set_padding)(EVP_CIPHER_CTX *, int);
+static int (*dlsym_EVP_CIPHER_CTX_test_flags)(const EVP_CIPHER_CTX *, int);
+static int (*dlsym_EVP_CIPHER_CTX_block_size)(const EVP_CIPHER_CTX *);
+#if OPENSSL_VERSION_NUMBER >= 0x10100000L
+static int (*dlsym_EVP_CIPHER_CTX_encrypting)(const EVP_CIPHER_CTX *);
+#endif
 static int (*dlsym_EVP_CipherInit_ex)(EVP_CIPHER_CTX *, const EVP_CIPHER *,  \
            ENGINE *, const unsigned char *, const unsigned char *, int);
 static int (*dlsym_EVP_CipherUpdate)(EVP_CIPHER_CTX *, unsigned char *,  \
@@ -46,6 +51,11 @@ typedef void (__cdecl *__dlsym_EVP_CIPHER_CTX_free)(EVP_CIPHER_CTX *);
 typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_cleanup)(EVP_CIPHER_CTX *);
 typedef void (__cdecl *__dlsym_EVP_CIPHER_CTX_init)(EVP_CIPHER_CTX *);
 typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_set_padding)(EVP_CIPHER_CTX *, int);
+typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_test_flags)(const EVP_CIPHER_CTX *, int);
+typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_block_size)(const EVP_CIPHER_CTX *);
+#if OPENSSL_VERSION_NUMBER >= 0x10100000L
+typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_encrypting)(const EVP_CIPHER_CTX *);
+#endif
 typedef int (__cdecl *__dlsym_EVP_CipherInit_ex)(EVP_CIPHER_CTX *,  \
              const EVP_CIPHER *, ENGINE *, const unsigned char *,  \
              const unsigned char *, int);
@@ -60,6 +70,11 @@ static __dlsym_EVP_CIPHER_CTX_free dlsym_EVP_CIPHER_CTX_free;
 static __dlsym_EVP_CIPHER_CTX_cleanup dlsym_EVP_CIPHER_CTX_cleanup;
 static __dlsym_EVP_CIPHER_CTX_init dlsym_EVP_CIPHER_CTX_init;
 static __dlsym_EVP_CIPHER_CTX_set_padding dlsym_EVP_CIPHER_CTX_set_padding;
+static __dlsym_EVP_CIPHER_CTX_test_flags dlsym_EVP_CIPHER_CTX_test_flags;
+static __dlsym_EVP_CIPHER_CTX_block_size dlsym_EVP_CIPHER_CTX_block_size;
+#if OPENSSL_VERSION_NUMBER >= 0x10100000L
+static __dlsym_EVP_CIPHER_CTX_encrypting dlsym_EVP_CIPHER_CTX_encrypting;
+#endif
 static __dlsym_EVP_CipherInit_ex dlsym_EVP_CipherInit_ex;
 static __dlsym_EVP_CipherUpdate dlsym_EVP_CipherUpdate;
 static __dlsym_EVP_CipherFinal_ex dlsym_EVP_CipherFinal_ex;
@@ -114,6 +129,14 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
                       "EVP_CIPHER_CTX_init");
   LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_set_padding, env, openssl,  \
                       "EVP_CIPHER_CTX_set_padding");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_test_flags, env, openssl,  \
+                      "EVP_CIPHER_CTX_test_flags");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_block_size, env, openssl,  \
+                      "EVP_CIPHER_CTX_block_size");
+#if OPENSSL_VERSION_NUMBER >= 0x10100000L
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_encrypting, env, openssl,  \
+                      "EVP_CIPHER_CTX_encrypting");
+#endif
   LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherInit_ex, env, openssl,  \
                       "EVP_CipherInit_ex");
   LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherUpdate, env, openssl,  \
@@ -135,6 +158,17 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
   LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_set_padding,  \
                       dlsym_EVP_CIPHER_CTX_set_padding, env,  \
                       openssl, "EVP_CIPHER_CTX_set_padding");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_test_flags,  \
+                      dlsym_EVP_CIPHER_CTX_test_flags, env,  \
+                      openssl, "EVP_CIPHER_CTX_test_flags");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_block_size,  \
+                      dlsym_EVP_CIPHER_CTX_block_size, env,  \
+                      openssl, "EVP_CIPHER_CTX_block_size");
+#if OPENSSL_VERSION_NUMBER >= 0x10100000L
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_encrypting,  \
+                      dlsym_EVP_CIPHER_CTX_encrypting, env,  \
+                      openssl, "EVP_CIPHER_CTX_encrypting");
+#endif
   LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherInit_ex, dlsym_EVP_CipherInit_ex,  \
                       env, openssl, "EVP_CipherInit_ex");
   LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherUpdate, dlsym_EVP_CipherUpdate,  \
@@ -253,14 +287,18 @@ JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_init
 static int check_update_max_output_len(EVP_CIPHER_CTX *context, int input_len, 
     int max_output_len)
 {
-  if (context->flags & EVP_CIPH_NO_PADDING) {
+  if (  dlsym_EVP_CIPHER_CTX_test_flags(context, EVP_CIPH_NO_PADDING) ) {
     if (max_output_len >= input_len) {
       return 1;
     }
     return 0;
   } else {
-    int b = context->cipher->block_size;
+    int b = dlsym_EVP_CIPHER_CTX_block_size(context);
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
     if (context->encrypt) {
+#else
+    if (dlsym_EVP_CIPHER_CTX_encrypting(context)) {
+#endif
       if (max_output_len >= input_len + b - 1) {
         return 1;
       }
@@ -307,10 +345,10 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_update
 static int check_doFinal_max_output_len(EVP_CIPHER_CTX *context, 
     int max_output_len)
 {
-  if (context->flags & EVP_CIPH_NO_PADDING) {
+  if (  dlsym_EVP_CIPHER_CTX_test_flags(context, EVP_CIPH_NO_PADDING) ) {
     return 1;
   } else {
-    int b = context->cipher->block_size;
+    int b = dlsym_EVP_CIPHER_CTX_block_size(context);
     if (max_output_len >= b) {
       return 1;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94ca52ae/hadoop-tools/hadoop-pipes/src/main/native/pipes/impl/HadoopPipes.cc
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-pipes/src/main/native/pipes/impl/HadoopPipes.cc b/hadoop-tools/hadoop-pipes/src/main/native/pipes/impl/HadoopPipes.cc
index 91fb5a4..45cb8c2 100644
--- a/hadoop-tools/hadoop-pipes/src/main/native/pipes/impl/HadoopPipes.cc
+++ b/hadoop-tools/hadoop-pipes/src/main/native/pipes/impl/HadoopPipes.cc
@@ -420,6 +420,7 @@ namespace HadoopPipes {
     }
 
     string createDigest(string &password, string& msg) {
+#if OPENSSL_VERSION_NUMBER < 0x10100000L
       HMAC_CTX ctx;
       unsigned char digest[EVP_MAX_MD_SIZE];
       HMAC_Init(&ctx, (const unsigned char *)password.c_str(), 
@@ -428,7 +429,16 @@ namespace HadoopPipes {
       unsigned int digestLen;
       HMAC_Final(&ctx, digest, &digestLen);
       HMAC_cleanup(&ctx);
-
+#else
+      HMAC_CTX *ctx = HMAC_CTX_new();
+      unsigned char digest[EVP_MAX_MD_SIZE];
+      HMAC_Init_ex(ctx, (const unsigned char *)password.c_str(),
+          password.length(), EVP_sha1(), NULL);
+      HMAC_Update(ctx, (const unsigned char *)msg.c_str(), msg.length());
+      unsigned int digestLen;
+      HMAC_Final(ctx, digest, &digestLen);
+      HMAC_CTX_free(ctx);
+#endif
       //now apply base64 encoding
       BIO *bmem, *b64;
       BUF_MEM *bptr;


---------------------------------------------------------------------
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: HADOOP-14455. ViewFileSystem#rename should support be supported within same nameservice with different mountpoints. Contributed by Brahma Reddy Battula.

Posted by in...@apache.org.
HADOOP-14455. ViewFileSystem#rename should support be supported within same nameservice with different mountpoints. Contributed by Brahma Reddy Battula.


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

Branch: refs/heads/HDFS-10467
Commit: 6d983cca52f113118bf49fec527ffb3eb869290a
Parents: 1a79dcf
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Jul 25 23:20:35 2017 +0800
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Jul 25 23:51:53 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/viewfs/Constants.java  |  2 +
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java | 79 +++++++++++-----
 .../org/apache/hadoop/fs/viewfs/ViewFs.java     | 43 ++++-----
 .../src/main/resources/core-default.xml         |  9 ++
 .../conf/TestCommonConfigurationFields.java     |  1 +
 .../hadoop/fs/contract/ContractTestUtils.java   | 54 +++++++++++
 .../fs/viewfs/ViewFileSystemBaseTest.java       | 79 +++++++++++++---
 .../apache/hadoop/fs/viewfs/ViewFsBaseTest.java | 94 ++++++++++++++++----
 .../fs/viewfs/TestViewFileSystemHdfs.java       | 22 +++++
 9 files changed, 309 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
index ec8ab2b..9882a8e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
@@ -66,4 +66,6 @@ public interface Constants {
 
   static public final FsPermission PERMISSION_555 =
       new FsPermission((short) 0555);
+
+  String CONFIG_VIEWFS_RENAME_STRATEGY = "fs.viewfs.rename.strategy";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
index 8265d89..158b099 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
@@ -126,7 +126,8 @@ public class ViewFileSystem extends FileSystem {
   Configuration config;
   InodeTree<FileSystem> fsState;  // the fs state; ie the mount table
   Path homeDir = null;
-  
+  // Default to rename within same mountpoint
+  private RenameStrategy renameStrategy = RenameStrategy.SAME_MOUNTPOINT;
   /**
    * Make the path Absolute and get the path-part of a pathname.
    * Checks that URI matches this file system 
@@ -207,6 +208,9 @@ public class ViewFileSystem extends FileSystem {
         }
       };
       workingDir = this.getHomeDirectory();
+      renameStrategy = RenameStrategy.valueOf(
+          conf.get(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+              RenameStrategy.SAME_MOUNTPOINT.toString()));
     } catch (URISyntaxException e) {
       throw new IOException("URISyntax exception: " + theUri);
     }
@@ -490,27 +494,55 @@ public class ViewFileSystem extends FileSystem {
     if (resDst.isInternalDir()) {
           throw readOnlyMountTable("rename", dst);
     }
-    /**
-    // Alternate 1: renames within same file system - valid but we disallow
-    // Alternate 2: (as described in next para - valid but we have disallowed it
-    //
-    // Note we compare the URIs. the URIs include the link targets. 
-    // hence we allow renames across mount links as long as the mount links
-    // point to the same target.
-    if (!resSrc.targetFileSystem.getUri().equals(
-              resDst.targetFileSystem.getUri())) {
-      throw new IOException("Renames across Mount points not supported");
-    }
-    */
-    
-    //
-    // Alternate 3 : renames ONLY within the the same mount links.
-    //
-    if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
-      throw new IOException("Renames across Mount points not supported");
+
+    URI srcUri = resSrc.targetFileSystem.getUri();
+    URI dstUri = resDst.targetFileSystem.getUri();
+
+    verifyRenameStrategy(srcUri, dstUri,
+        resSrc.targetFileSystem == resDst.targetFileSystem, renameStrategy);
+
+    ChRootedFileSystem srcFS = (ChRootedFileSystem) resSrc.targetFileSystem;
+    ChRootedFileSystem dstFS = (ChRootedFileSystem) resDst.targetFileSystem;
+    return srcFS.getMyFs().rename(srcFS.fullPath(resSrc.remainingPath),
+        dstFS.fullPath(resDst.remainingPath));
+  }
+
+  static void verifyRenameStrategy(URI srcUri, URI dstUri,
+      boolean isSrcDestSame, ViewFileSystem.RenameStrategy renameStrategy)
+      throws IOException {
+    switch (renameStrategy) {
+    case SAME_FILESYSTEM_ACROSS_MOUNTPOINT:
+      if (srcUri.getAuthority() != null) {
+        if (!(srcUri.getScheme().equals(dstUri.getScheme()) && srcUri
+            .getAuthority().equals(dstUri.getAuthority()))) {
+          throw new IOException("Renames across Mount points not supported");
+        }
+      }
+
+      break;
+    case SAME_TARGET_URI_ACROSS_MOUNTPOINT:
+      // Alternate 2: Rename across mountpoints with same target.
+      // i.e. Rename across alias mountpoints.
+      //
+      // Note we compare the URIs. the URIs include the link targets.
+      // hence we allow renames across mount links as long as the mount links
+      // point to the same target.
+      if (!srcUri.equals(dstUri)) {
+        throw new IOException("Renames across Mount points not supported");
+      }
+
+      break;
+    case SAME_MOUNTPOINT:
+      //
+      // Alternate 3 : renames ONLY within the the same mount links.
+      //
+      if (!isSrcDestSame) {
+        throw new IOException("Renames across Mount points not supported");
+      }
+      break;
+    default:
+      throw new IllegalArgumentException ("Unexpected rename strategy");
     }
-    return resSrc.targetFileSystem.rename(resSrc.remainingPath,
-        resDst.remainingPath);
   }
 
   @Override
@@ -1241,4 +1273,9 @@ public class ViewFileSystem extends FileSystem {
       return allPolicies;
     }
   }
+
+  enum RenameStrategy {
+    SAME_MOUNTPOINT, SAME_TARGET_URI_ACROSS_MOUNTPOINT,
+    SAME_FILESYSTEM_ACROSS_MOUNTPOINT
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 3a34a91..364485f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -157,7 +157,9 @@ public class ViewFs extends AbstractFileSystem {
   final Configuration config;
   InodeTree<AbstractFileSystem> fsState;  // the fs state; ie the mount table
   Path homeDir = null;
-  
+  private ViewFileSystem.RenameStrategy renameStrategy =
+      ViewFileSystem.RenameStrategy.SAME_MOUNTPOINT;
+
   static AccessControlException readOnlyMountTable(final String operation,
       final String p) {
     return new AccessControlException( 
@@ -237,6 +239,9 @@ public class ViewFs extends AbstractFileSystem {
         // return MergeFs.createMergeFs(mergeFsURIList, config);
       }
     };
+    renameStrategy = ViewFileSystem.RenameStrategy.valueOf(
+        conf.get(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+            ViewFileSystem.RenameStrategy.SAME_MOUNTPOINT.toString()));
   }
 
   @Override
@@ -495,37 +500,23 @@ public class ViewFs extends AbstractFileSystem {
               + " is readOnly");
     }
 
-    InodeTree.ResolveResult<AbstractFileSystem> resDst = 
+    InodeTree.ResolveResult<AbstractFileSystem> resDst =
                                 fsState.resolve(getUriPath(dst), false);
     if (resDst.isInternalDir()) {
       throw new AccessControlException(
           "Cannot Rename within internal dirs of mount table: dest=" + dst
               + " is readOnly");
     }
-    
-    /**
-    // Alternate 1: renames within same file system - valid but we disallow
-    // Alternate 2: (as described in next para - valid but we have disallowed it
-    //
-    // Note we compare the URIs. the URIs include the link targets. 
-    // hence we allow renames across mount links as long as the mount links
-    // point to the same target.
-    if (!resSrc.targetFileSystem.getUri().equals(
-              resDst.targetFileSystem.getUri())) {
-      throw new IOException("Renames across Mount points not supported");
-    }
-    */
-    
-    //
-    // Alternate 3 : renames ONLY within the the same mount links.
-    //
-
-    if (resSrc.targetFileSystem !=resDst.targetFileSystem) {
-      throw new IOException("Renames across Mount points not supported");
-    }
-    
-    resSrc.targetFileSystem.renameInternal(resSrc.remainingPath,
-      resDst.remainingPath, overwrite);
+    //Alternate 1: renames within same file system
+    URI srcUri = resSrc.targetFileSystem.getUri();
+    URI dstUri = resDst.targetFileSystem.getUri();
+    ViewFileSystem.verifyRenameStrategy(srcUri, dstUri,
+        resSrc.targetFileSystem == resDst.targetFileSystem, renameStrategy);
+
+    ChRootedFs srcFS = (ChRootedFs) resSrc.targetFileSystem;
+    ChRootedFs dstFS = (ChRootedFs) resDst.targetFileSystem;
+    srcFS.getMyFs().renameInternal(srcFS.fullPath(resSrc.remainingPath),
+        dstFS.fullPath(resDst.remainingPath), overwrite);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/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 d5ddc7f..593fd85 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
@@ -801,6 +801,15 @@
 </property>
 
 <property>
+  <name>fs.viewfs.rename.strategy</name>
+  <value>SAME_MOUNTPOINT</value>
+  <description>Allowed rename strategy to rename between multiple mountpoints.
+    Allowed values are SAME_MOUNTPOINT,SAME_TARGET_URI_ACROSS_MOUNTPOINT and
+    SAME_FILESYSTEM_ACROSS_MOUNTPOINT.
+  </description>
+</property>
+
+<property>
   <name>fs.AbstractFileSystem.ftp.impl</name>
   <value>org.apache.hadoop.fs.ftp.FtpFs</value>
   <description>The FileSystem for Ftp: uris.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index ef74cba..da37e68 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -95,6 +95,7 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPropsToSkipCompare.add("nfs3.mountd.port");
     xmlPropsToSkipCompare.add("nfs3.server.port");
     xmlPropsToSkipCompare.add("test.fs.s3n.name");
+    xmlPropsToSkipCompare.add("fs.viewfs.rename.strategy");
 
     // S3N/S3A properties are in a different subtree.
     // - org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
index 39c6d18..e60fd43 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.contract;
 
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -718,6 +719,21 @@ public class ContractTestUtils extends Assert {
   /**
    * Assert that a file exists and whose {@link FileStatus} entry
    * declares that this is a file and not a symlink or directory.
+   *
+   * @param fileContext filesystem to resolve path against
+   * @param filename    name of the file
+   * @throws IOException IO problems during file operations
+   */
+  public static void assertIsFile(FileContext fileContext, Path filename)
+      throws IOException {
+    assertPathExists(fileContext, "Expected file", filename);
+    FileStatus status = fileContext.getFileStatus(filename);
+    assertIsFile(filename, status);
+  }
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
    * @param filename name of the file
    * @param status file status
    */
@@ -766,6 +782,25 @@ public class ContractTestUtils extends Assert {
   }
 
   /**
+   * Assert that a path exists -but make no assertions as to the
+   * type of that entry.
+   *
+   * @param fileContext fileContext to examine
+   * @param message     message to include in the assertion failure message
+   * @param path        path in the filesystem
+   * @throws FileNotFoundException raised if the path is missing
+   * @throws IOException           IO problems
+   */
+  public static void assertPathExists(FileContext fileContext, String message,
+      Path path) throws IOException {
+    if (!fileContext.util().exists(path)) {
+      //failure, report it
+      throw new FileNotFoundException(
+          message + ": not found " + path + " in " + path.getParent());
+    }
+  }
+
+  /**
    * Assert that a path does not exist.
    *
    * @param fileSystem filesystem to examine
@@ -786,6 +821,25 @@ public class ContractTestUtils extends Assert {
   }
 
   /**
+   * Assert that a path does not exist.
+   *
+   * @param fileContext fileContext to examine
+   * @param message     message to include in the assertion failure message
+   * @param path        path in the filesystem
+   * @throws IOException IO problems
+   */
+  public static void assertPathDoesNotExist(FileContext fileContext,
+      String message, Path path) throws IOException {
+    try {
+      FileStatus status = fileContext.getFileStatus(path);
+      fail(message + ": unexpectedly found " + path + " as  " + status);
+    } catch (FileNotFoundException expected) {
+      //this is expected
+
+    }
+  }
+
+  /**
    * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry.
    * @param fs filesystem
    * @param dir directory to scan

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
index 68a7560..db2d2d7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclUtil;
@@ -51,6 +52,7 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assume;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -366,28 +368,83 @@ abstract public class ViewFileSystemBaseTest {
   }
   
   // rename across mount points that point to same target also fail 
-  @Test(expected=IOException.class) 
+  @Test
   public void testRenameAcrossMounts1() throws IOException {
     fileSystemTestHelper.createFile(fsView, "/user/foo");
-    fsView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
-    /* - code if we had wanted this to succeed
-    Assert.assertFalse(fSys.exists(new Path("/user/foo")));
-    Assert.assertFalse(fSysLocal.exists(new Path(targetTestRoot,"user/foo")));
-    Assert.assertTrue(fSys.isFile(FileSystemTestHelper.getTestRootPath(fSys,"/user2/fooBarBar")));
-    Assert.assertTrue(fSysLocal.isFile(new Path(targetTestRoot,"user/fooBarBar")));
-    */
+    try {
+      fsView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
+      ContractTestUtils.fail("IOException is not thrown on rename operation");
+    } catch (IOException e) {
+      GenericTestUtils
+          .assertExceptionContains("Renames across Mount points not supported",
+              e);
+    }
   }
   
   
   // rename across mount points fail if the mount link targets are different
   // even if the targets are part of the same target FS
 
-  @Test(expected=IOException.class) 
+  @Test
   public void testRenameAcrossMounts2() throws IOException {
     fileSystemTestHelper.createFile(fsView, "/user/foo");
-    fsView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+    try {
+      fsView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+      ContractTestUtils.fail("IOException is not thrown on rename operation");
+    } catch (IOException e) {
+      GenericTestUtils
+          .assertExceptionContains("Renames across Mount points not supported",
+              e);
+    }
   }
-  
+
+  // RenameStrategy SAME_TARGET_URI_ACROSS_MOUNTPOINT enabled
+  // to rename across mount points that point to same target URI
+  @Test
+  public void testRenameAcrossMounts3() throws IOException {
+    Configuration conf2 = new Configuration(conf);
+    conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+        ViewFileSystem.RenameStrategy.SAME_TARGET_URI_ACROSS_MOUNTPOINT
+            .toString());
+    FileSystem fsView2 = FileSystem.newInstance(FsConstants.VIEWFS_URI, conf2);
+    fileSystemTestHelper.createFile(fsView2, "/user/foo");
+    fsView2.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
+    ContractTestUtils
+        .assertPathDoesNotExist(fsView2, "src should not exist after rename",
+            new Path("/user/foo"));
+    ContractTestUtils
+        .assertPathDoesNotExist(fsTarget, "src should not exist after rename",
+            new Path(targetTestRoot, "user/foo"));
+    ContractTestUtils.assertIsFile(fsView2,
+        fileSystemTestHelper.getTestRootPath(fsView2, "/user2/fooBarBar"));
+    ContractTestUtils
+        .assertIsFile(fsTarget, new Path(targetTestRoot, "user/fooBarBar"));
+  }
+
+  // RenameStrategy SAME_FILESYSTEM_ACROSS_MOUNTPOINT enabled
+  // to rename across mount points where the mount link targets are different
+  // but are part of the same target FS
+  @Test
+  public void testRenameAcrossMounts4() throws IOException {
+    Configuration conf2 = new Configuration(conf);
+    conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+        ViewFileSystem.RenameStrategy.SAME_FILESYSTEM_ACROSS_MOUNTPOINT
+            .toString());
+    FileSystem fsView2 = FileSystem.newInstance(FsConstants.VIEWFS_URI, conf2);
+    fileSystemTestHelper.createFile(fsView2, "/user/foo");
+    fsView2.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+    ContractTestUtils
+        .assertPathDoesNotExist(fsView2, "src should not exist after rename",
+            new Path("/user/foo"));
+    ContractTestUtils
+        .assertPathDoesNotExist(fsTarget, "src should not exist after rename",
+            new Path(targetTestRoot, "user/foo"));
+    ContractTestUtils.assertIsFile(fsView2,
+        fileSystemTestHelper.getTestRootPath(fsView2, "/data/fooBar"));
+    ContractTestUtils
+        .assertIsFile(fsTarget, new Path(targetTestRoot, "data/fooBar"));
+  }
+
   static protected boolean SupportsBlocks = false; //  local fs use 1 block
                                                    // override for HDFS
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
index fdc6389..d72ab74 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.local.LocalConfigKeys;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -66,6 +67,7 @@ import org.apache.hadoop.fs.viewfs.ViewFs.MountPoint;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -345,33 +347,93 @@ abstract public class ViewFsBaseTest {
   }
   
   // rename across mount points that point to same target also fail 
-  @Test(expected=IOException.class) 
+  @Test
   public void testRenameAcrossMounts1() throws IOException {
     fileContextTestHelper.createFile(fcView, "/user/foo");
-    fcView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
-    /* - code if we had wanted this to succeed
-    Assert.assertFalse(exists(fc, new Path("/user/foo")));
-    Assert.assertFalse(exists(fclocal, new Path(targetTestRoot,"user/foo")));
-    Assert.assertTrue(isFile(fc,
-       FileContextTestHelper.getTestRootPath(fc,"/user2/fooBarBar")));
-    Assert.assertTrue(isFile(fclocal,
-        new Path(targetTestRoot,"user/fooBarBar")));
-    */
+    try {
+      fcView.rename(new Path("/user/foo"), new Path("/user2/fooBarBar"));
+      ContractTestUtils.fail("IOException is not thrown on rename operation");
+    } catch (IOException e) {
+      GenericTestUtils
+          .assertExceptionContains("Renames across Mount points not supported",
+              e);
+    }
   }
   
   
   // rename across mount points fail if the mount link targets are different
   // even if the targets are part of the same target FS
 
-  @Test(expected=IOException.class) 
+  @Test
   public void testRenameAcrossMounts2() throws IOException {
     fileContextTestHelper.createFile(fcView, "/user/foo");
-    fcView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+    try {
+      fcView.rename(new Path("/user/foo"), new Path("/data/fooBar"));
+      ContractTestUtils.fail("IOException is not thrown on rename operation");
+    } catch (IOException e) {
+      GenericTestUtils
+          .assertExceptionContains("Renames across Mount points not supported",
+              e);
+    }
   }
-  
-  
-  
-  
+
+  // RenameStrategy SAME_TARGET_URI_ACROSS_MOUNTPOINT enabled
+  // to rename across mount points that point to same target URI
+  @Test
+  public void testRenameAcrossMounts3() throws IOException {
+    Configuration conf2 = new Configuration(conf);
+    conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+        ViewFileSystem.RenameStrategy.SAME_TARGET_URI_ACROSS_MOUNTPOINT
+            .toString());
+
+    FileContext fcView2 =
+        FileContext.getFileContext(FsConstants.VIEWFS_URI, conf2);
+    String user1Path = "/user/foo";
+    fileContextTestHelper.createFile(fcView2, user1Path);
+    String user2Path = "/user2/fooBarBar";
+    Path user2Dst = new Path(user2Path);
+    fcView2.rename(new Path(user1Path), user2Dst);
+    ContractTestUtils
+        .assertPathDoesNotExist(fcView2, "src should not exist after rename",
+            new Path(user1Path));
+    ContractTestUtils
+        .assertPathDoesNotExist(fcTarget, "src should not exist after rename",
+            new Path(targetTestRoot, "user/foo"));
+    ContractTestUtils.assertIsFile(fcView2,
+        fileContextTestHelper.getTestRootPath(fcView2, user2Path));
+    ContractTestUtils
+        .assertIsFile(fcTarget, new Path(targetTestRoot, "user/fooBarBar"));
+  }
+
+  // RenameStrategy SAME_FILESYSTEM_ACROSS_MOUNTPOINT enabled
+  // to rename across mount points if the mount link targets are different
+  // but are part of the same target FS
+  @Test
+  public void testRenameAcrossMounts4() throws IOException {
+    Configuration conf2 = new Configuration(conf);
+    conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY,
+        ViewFileSystem.RenameStrategy.SAME_FILESYSTEM_ACROSS_MOUNTPOINT
+            .toString());
+    FileContext fcView2 =
+        FileContext.getFileContext(FsConstants.VIEWFS_URI, conf2);
+    String userPath = "/user/foo";
+    fileContextTestHelper.createFile(fcView2, userPath);
+    String anotherMountPath = "/data/fooBar";
+    Path anotherDst = new Path(anotherMountPath);
+    fcView2.rename(new Path(userPath), anotherDst);
+
+    ContractTestUtils
+        .assertPathDoesNotExist(fcView2, "src should not exist after rename",
+            new Path(userPath));
+    ContractTestUtils
+        .assertPathDoesNotExist(fcTarget, "src should not exist after rename",
+            new Path(targetTestRoot, "user/foo"));
+    ContractTestUtils.assertIsFile(fcView2,
+        fileContextTestHelper.getTestRootPath(fcView2, anotherMountPath));
+    ContractTestUtils
+        .assertIsFile(fcView2, new Path(targetTestRoot, "data/fooBar"));
+  }
+
   static protected boolean SupportsBlocks = false; //  local fs use 1 block
                                                    // override for HDFS
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d983cca/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
index 58b77f6..b8f5379 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -247,4 +249,24 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     Assert.assertTrue("File checksum not matching!",
         fileChecksumViaViewFs.equals(fileChecksumViaTargetFs));
   }
+
+  //Rename should fail on across different fileSystems
+  @Test
+  public void testRenameAccorssFilesystem() throws IOException {
+    //data is mountpoint in nn1
+    Path mountDataRootPath = new Path("/data");
+    //mountOnNn2 is nn2 mountpoint
+    Path fsTargetFilePath = new Path("/mountOnNn2");
+    Path filePath = new Path(mountDataRootPath + "/ttest");
+    Path hdfFilepath = new Path(fsTargetFilePath + "/ttest2");
+    fsView.create(filePath);
+    try {
+      fsView.rename(filePath, hdfFilepath);
+      ContractTestUtils.fail("Should thrown IOE on Renames across filesytems");
+    } catch (IOException e) {
+      GenericTestUtils
+          .assertExceptionContains("Renames across Mount points not supported",
+              e);
+    }
+  }
 }


---------------------------------------------------------------------
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: YARN-6307. Refactor FairShareComparator#compare (Contributed by Yufei Gu via Daniel Templeton)

Posted by in...@apache.org.
YARN-6307. Refactor FairShareComparator#compare (Contributed by Yufei Gu via Daniel Templeton)


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

Branch: refs/heads/HDFS-10467
Commit: f81a4efb8c40f99a9a6b7b42d3b6eeedf43eb27a
Parents: ac9489f
Author: Daniel Templeton <te...@apache.org>
Authored: Tue Jul 25 13:00:31 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Tue Jul 25 13:00:31 2017 -0700

----------------------------------------------------------------------
 .../fair/policies/FairSharePolicy.java          | 125 ++++++++++++-------
 1 file changed, 81 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f81a4efb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.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/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 2a852aa..0ef90a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -56,24 +56,28 @@ public class FairSharePolicy extends SchedulingPolicy {
   }
 
   /**
-   * Compare Schedulables via weighted fair sharing. In addition, Schedulables
-   * below their min share get priority over those whose min share is met.
+   * Compare Schedulables mainly via fair share usage to meet fairness.
+   * Specifically, it goes through following four steps.
    *
-   * Schedulables without resource demand get lower priority than
-   * ones who have demands.
+   * 1. Compare demands. Schedulables without resource demand get lower priority
+   * than ones who have demands.
    * 
-   * Schedulables below their min share are compared by how far below it they
-   * are as a ratio. For example, if job A has 8 out of a min share of 10 tasks
-   * and job B has 50 out of a min share of 100, then job B is scheduled next,
-   * because B is at 50% of its min share and A is at 80% of its min share.
+   * 2. Compare min share usage. Schedulables below their min share are compared
+   * by how far below it they are as a ratio. For example, if job A has 8 out
+   * of a min share of 10 tasks and job B has 50 out of a min share of 100,
+   * then job B is scheduled next, because B is at 50% of its min share and A
+   * is at 80% of its min share.
    * 
-   * Schedulables above their min share are compared by (runningTasks / weight).
+   * 3. Compare fair share usage. Schedulables above their min share are
+   * compared by fair share usage by checking (resource usage / weight).
    * If all weights are equal, slots are given to the job with the fewest tasks;
    * otherwise, jobs with more weight get proportionally more slots. If weight
    * equals to 0, we can't compare Schedulables by (resource usage/weight).
    * There are two situations: 1)All weights equal to 0, slots are given
    * to one with less resource usage. 2)Only one of weight equals to 0, slots
    * are given to the one with non-zero weight.
+   *
+   * 4. Break the tie by compare submit time and job name.
    */
   private static class FairShareComparator implements Comparator<Schedulable>,
       Serializable {
@@ -82,37 +86,88 @@ public class FairSharePolicy extends SchedulingPolicy {
 
     @Override
     public int compare(Schedulable s1, Schedulable s2) {
+      int res = compareDemand(s1, s2);
+
+      // Pre-compute resource usages to avoid duplicate calculation
+      Resource resourceUsage1 = s1.getResourceUsage();
+      Resource resourceUsage2 = s2.getResourceUsage();
+
+      if (res == 0) {
+        res = compareMinShareUsage(s1, s2, resourceUsage1, resourceUsage2);
+      }
+
+      if (res == 0) {
+        res = compareFairShareUsage(s1, s2, resourceUsage1, resourceUsage2);
+      }
+
+      // Break the tie by submit time
+      if (res == 0) {
+        res = (int) Math.signum(s1.getStartTime() - s2.getStartTime());
+      }
+
+      // Break the tie by job name
+      if (res == 0) {
+        res = s1.getName().compareTo(s2.getName());
+      }
+
+      return res;
+    }
+
+    private int compareDemand(Schedulable s1, Schedulable s2) {
+      int res = 0;
       Resource demand1 = s1.getDemand();
       Resource demand2 = s2.getDemand();
       if (demand1.equals(Resources.none()) && Resources.greaterThan(
           RESOURCE_CALCULATOR, null, demand2, Resources.none())) {
-        return 1;
+        res = 1;
       } else if (demand2.equals(Resources.none()) && Resources.greaterThan(
           RESOURCE_CALCULATOR, null, demand1, Resources.none())) {
-        return -1;
+        res = -1;
       }
+      return res;
+    }
 
-      double minShareRatio1, minShareRatio2;
-      double useToWeightRatio1, useToWeightRatio2;
-      double weight1, weight2;
-      //Do not repeat the getResourceUsage calculation
-      Resource resourceUsage1 = s1.getResourceUsage();
-      Resource resourceUsage2 = s2.getResourceUsage();
+    private int compareMinShareUsage(Schedulable s1, Schedulable s2,
+        Resource resourceUsage1, Resource resourceUsage2) {
+      int res;
       Resource minShare1 = Resources.min(RESOURCE_CALCULATOR, null,
-          s1.getMinShare(), demand1);
+          s1.getMinShare(), s1.getDemand());
       Resource minShare2 = Resources.min(RESOURCE_CALCULATOR, null,
-          s2.getMinShare(), demand2);
+          s2.getMinShare(), s2.getDemand());
       boolean s1Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
           resourceUsage1, minShare1);
       boolean s2Needy = Resources.lessThan(RESOURCE_CALCULATOR, null,
           resourceUsage2, minShare2);
-      minShareRatio1 = (double) resourceUsage1.getMemorySize()
-          / Resources.max(RESOURCE_CALCULATOR, null, minShare1, ONE).getMemorySize();
-      minShareRatio2 = (double) resourceUsage2.getMemorySize()
-          / Resources.max(RESOURCE_CALCULATOR, null, minShare2, ONE).getMemorySize();
 
-      weight1 = s1.getWeights().getWeight(ResourceType.MEMORY);
-      weight2 = s2.getWeights().getWeight(ResourceType.MEMORY);
+      if (s1Needy && !s2Needy) {
+        res = -1;
+      } else if (s2Needy && !s1Needy) {
+        res = 1;
+      } else if (s1Needy && s2Needy) {
+        double minShareRatio1 = (double) resourceUsage1.getMemorySize() /
+            Resources.max(RESOURCE_CALCULATOR, null, minShare1, ONE)
+                .getMemorySize();
+        double minShareRatio2 = (double) resourceUsage2.getMemorySize() /
+            Resources.max(RESOURCE_CALCULATOR, null, minShare2, ONE)
+                .getMemorySize();
+        res = (int) Math.signum(minShareRatio1 - minShareRatio2);
+      } else {
+        res = 0;
+      }
+
+      return res;
+    }
+
+    /**
+     * To simplify computation, use weights instead of fair shares to calculate
+     * fair share usage.
+     */
+    private int compareFairShareUsage(Schedulable s1, Schedulable s2,
+        Resource resourceUsage1, Resource resourceUsage2) {
+      double weight1 = s1.getWeights().getWeight(ResourceType.MEMORY);
+      double weight2 = s2.getWeights().getWeight(ResourceType.MEMORY);
+      double useToWeightRatio1;
+      double useToWeightRatio2;
       if (weight1 > 0.0 && weight2 > 0.0) {
         useToWeightRatio1 = resourceUsage1.getMemorySize() / weight1;
         useToWeightRatio2 = resourceUsage2.getMemorySize() / weight2;
@@ -130,25 +185,7 @@ public class FairSharePolicy extends SchedulingPolicy {
         }
       }
 
-      int res = 0;
-      if (s1Needy && !s2Needy)
-        res = -1;
-      else if (s2Needy && !s1Needy)
-        res = 1;
-      else if (s1Needy && s2Needy)
-        res = (int) Math.signum(minShareRatio1 - minShareRatio2);
-      else
-        // Neither schedulable is needy
-        res = (int) Math.signum(useToWeightRatio1 - useToWeightRatio2);
-      if (res == 0) {
-        // Apps are tied in fairness ratio. Break the tie by submit time and job
-        // name to get a deterministic ordering, which is useful for unit tests.
-        res = (int) Math.signum(s1.getStartTime() - s2.getStartTime());
-        if (res == 0) {
-          res = s1.getName().compareTo(s2.getName());
-        }
-      }
-      return res;
+      return (int) Math.signum(useToWeightRatio1 - useToWeightRatio2);
     }
   }
 


---------------------------------------------------------------------
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: HDFS-12176. dfsadmin shows DFS Used%: NaN% if the cluster has zero block. Contributed by Weiwei Yang.

Posted by in...@apache.org.
HDFS-12176. dfsadmin shows DFS Used%: NaN% if the cluster has zero block. 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/770cc462
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/770cc462
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/770cc462

Branch: refs/heads/HDFS-10467
Commit: 770cc462281518545e3d1c0f8c21cf9ec9673200
Parents: e315328
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jul 24 16:23:01 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jul 24 16:24:23 2017 +0900

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/770cc462/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index ea76093..f2233eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -520,9 +520,13 @@ public class DFSAdmin extends FsShell {
         + " (" + StringUtils.byteDesc(remaining) + ")");
     System.out.println("DFS Used: " + used
                        + " (" + StringUtils.byteDesc(used) + ")");
+    double dfsUsedPercent = 0;
+    if (presentCapacity != 0) {
+      dfsUsedPercent = used/(double)presentCapacity;
+    }
     System.out.println("DFS Used%: "
-        + StringUtils.formatPercent(used/(double)presentCapacity, 2));
-    
+        + StringUtils.formatPercent(dfsUsedPercent, 2));
+
     /* These counts are not always upto date. They are updated after  
      * iteration of an internal list. Should be updated in a few seconds to 
      * minutes. Use "-metaSave" to list of all such blocks and accurate 


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletSpec.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletSpec.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletSpec.java
new file mode 100644
index 0000000..8aeba93
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/HamletSpec.java
@@ -0,0 +1,3101 @@
+/**
+* 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.webapp.hamlet2;
+
+import java.lang.annotation.*;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+/**
+ * HTML5 compatible HTML4 builder interfaces.
+ *
+ * <p>Generated from HTML 4.01 strict DTD and HTML5 diffs.
+ * <br>cf. http://www.w3.org/TR/html4/
+ * <br>cf. http://www.w3.org/TR/html5-diff/
+ * <p> The omitted attributes and elements (from the 4.01 DTD)
+ * are for HTML5 compatibility.
+ *
+ * <p>Note, the common argument selector uses the same syntax as Haml/Sass:
+ * <pre>  selector ::= (#id)?(.class)*</pre>
+ * cf. http://haml-lang.com/
+ *
+ * <p>The naming convention used in this class is slightly different from
+ * normal classes. A CamelCase interface corresponds to an entity in the DTD.
+ * _CamelCase is for internal refactoring. An element builder interface is in
+ * UPPERCASE, corresponding to an element definition in the DTD. $lowercase is
+ * used as attribute builder methods to differentiate from element builder
+ * methods.
+ */
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+public class HamletSpec {
+  // The enum values are lowercase for better compression,
+  // while avoiding runtime conversion.
+  // cf. http://www.w3.org/Protocols/HTTP/Performance/Compression/HTMLCanon.html
+  //     http://www.websiteoptimization.com/speed/tweak/lowercase/
+  /** %Shape (case-insensitive) */
+  public enum Shape {
+    /**
+     * rectangle
+     */
+    rect,
+    /**
+     * circle
+     */
+    circle,
+    /**
+     * polygon
+     */
+    poly,
+    /**
+     * default
+     */
+    Default
+  };
+
+  /** Values for the %18n dir attribute (case-insensitive) */
+  public enum Dir {
+    /**
+     * left to right
+     */
+    ltr,
+    /**
+     * right to left
+     */
+    rtl
+  };
+
+  /** %MediaDesc (case-sensitive) */
+  public enum Media {
+    /**
+     * computer screen
+     */
+    screen,
+    /**
+     * teletype/terminal
+     */
+    tty,
+    /**
+     * television
+     */
+    tv,
+    /**
+     * projection
+     */
+    projection,
+    /**
+     * mobile device
+     */
+    handheld,
+    /**
+     * print media
+     */
+    print,
+    /**
+     * braille
+     */
+    braille,
+    /**
+     * aural
+     */
+    aural,
+    /**
+     * suitable all media
+     */
+    all
+  };
+
+  /** %LinkTypes (case-insensitive) */
+  public enum LinkType {
+    /**
+     *
+     */
+    alternate,
+    /**
+     *
+     */
+    stylesheet,
+    /**
+     *
+     */
+    start,
+    /**
+     *
+     */
+    next,
+    /**
+     *
+     */
+    prev,
+    /**
+     *
+     */
+    contents,
+    /**
+     *
+     */
+    index,
+    /**
+     *
+     */
+    glossary,
+    /**
+     *
+     */
+    copyright,
+    /**
+     *
+     */
+    chapter,
+    /**
+     *
+     */
+    section,
+    /**
+     *
+     */
+    subsection,
+    /**
+     *
+     */
+    appendix,
+    /**
+     *
+     */
+    help,
+    /**
+     *
+     */
+    bookmark
+  };
+
+  /** Values for form methods (case-insensitive) */
+  public enum Method {
+    /**
+     * HTTP GET
+     */
+    get,
+    /**
+     * HTTP POST
+     */
+    post
+  };
+
+  /** %InputType (case-insensitive) */
+  public enum InputType {
+    /**
+     *
+     */
+    text,
+    /**
+     *
+     */
+    password,
+    /**
+     *
+     */
+    checkbox,
+    /**
+     *
+     */
+    radio,
+    /**
+     *
+     */
+    submit,
+    /**
+     *
+     */
+    reset,
+    /**
+     *
+     */
+    file,
+    /**
+     *
+     */
+    hidden,
+    /**
+     *
+     */
+    image,
+    /**
+     *
+     */
+    button
+  };
+
+  /** Values for button types */
+  public enum ButtonType {
+    /**
+     *
+     */
+    button,
+    /**
+     *
+     */
+    submit,
+    /**
+     *
+     */
+    reset
+  };
+
+  /** %Scope (case-insensitive) */
+  public enum Scope {
+    /**
+     *
+     */
+    row,
+    /**
+     *
+     */
+    col,
+    /**
+     *
+     */
+    rowgroup,
+    /**
+     *
+     */
+    colgroup
+  };
+
+  /**
+   * The element annotation for specifying element options other than
+   * attributes and allowed child elements
+   */
+  @Target({ElementType.TYPE})
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Element {
+    /**
+     * Whether the start tag is required for the element.
+     * @return true if start tag is required
+     */
+    boolean startTag() default true;
+
+    /**
+     * Whether the end tag is required.
+     * @return true if end tag is required
+     */
+    boolean endTag() default true;
+  }
+
+  /**
+   *
+   */
+  public interface __ {}
+
+  /**
+   *
+   */
+  public interface _Child extends __ {
+    /**
+     * Finish the current element.
+     * @return the parent element
+     */
+    __ __();
+  }
+
+  /**
+   *
+   */
+  public interface _Script {
+    /**
+     * Add a script element.
+     * @return a script element builder
+     */
+    SCRIPT script();
+
+    /**
+     * Add a script element
+     * @param src uri of the script
+     * @return the current element builder
+     */
+    _Script script(String src);
+  }
+
+  /**
+   *
+   */
+  public interface _Object {
+      /**
+     * Add an object element.
+     * @return an object element builder
+     */
+    OBJECT object();
+
+    /**
+     * Add an object element.
+     * @param selector as #id.class etc.
+     * @return an object element builder
+     */
+    OBJECT object(String selector);
+  }
+
+  /** %head.misc */
+  public interface HeadMisc extends _Script, _Object {
+    /**
+     * Add a style element.
+     * @return a style element builder
+     */
+    STYLE style();
+
+    /**
+     * Add a css style element.
+     * @param lines content of the style sheet
+     * @return the current element builder
+     */
+    HeadMisc style(Object... lines);
+
+    /**
+     * Add a meta element.
+     * @return a meta element builder
+     */
+    META meta();
+
+    /**
+     * Add a meta element.
+     * Shortcut of <code>meta().$name(name).$content(content).__();</code>
+     * @param name of the meta element
+     * @param content of the meta element
+     * @return the current element builder
+     */
+    HeadMisc meta(String name, String content);
+
+    /**
+     * Add a meta element with http-equiv attribute.
+     * Shortcut of <br>
+     * <code>meta().$http_equiv(header).$content(content).__();</code>
+     * @param header for the http-equiv attribute
+     * @param content of the header
+     * @return the current element builder
+     */
+    HeadMisc meta_http(String header, String content);
+
+    /**
+     * Add a link element.
+     * @return a link element builder
+     */
+    LINK link();
+
+    /**
+     * Add a link element.
+     * Implementation should try to figure out type by the suffix of href.
+     * So <code>link("style.css");</code> is a shortcut of
+     * <code>link().$rel("stylesheet").$type("text/css").$href("style.css").__();
+     * </code>
+     * @param href of the link
+     * @return the current element builder
+     */
+    HeadMisc link(String href);
+  }
+
+  /** %heading */
+  public interface Heading {
+    /**
+     * Add an H1 element.
+     * @return a new H1 element builder
+     */
+    H1 h1();
+
+    /**
+     * Add a complete H1 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h1(String cdata);
+
+    /**
+     * Add a complete H1 element
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h1(String selector, String cdata);
+
+    /**
+     * Add an H2 element.
+     * @return a new H2 element builder
+     */
+    H2 h2();
+
+    /**
+     * Add a complete H2 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h2(String cdata);
+
+    /**
+     * Add a complete H1 element
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h2(String selector, String cdata);
+
+    /**
+     * Add an H3 element.
+     * @return a new H3 element builder
+     */
+    H3 h3();
+
+    /**
+     * Add a complete H3 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h3(String cdata);
+
+    /**
+     * Add a complete H1 element
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h3(String selector, String cdata);
+
+    /**
+     * Add an H4 element.
+     * @return a new H4 element builder
+     */
+    H4 h4();
+
+    /**
+     * Add a complete H4 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h4(String cdata);
+
+    /**
+     * Add a complete H4 element
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h4(String selector, String cdata);
+
+    /**
+     * Add an H5 element.
+     * @return a new H5 element builder
+     */
+    H5 h5();
+
+    /**
+     * Add a complete H5 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h5(String cdata);
+
+    /**
+     * Add a complete H5 element
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h5(String selector, String cdata);
+
+    /**
+     * Add an H6 element.
+     * @return a new H6 element builder
+     */
+    H6 h6();
+
+    /**
+     * Add a complete H6 element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h6(String cdata);
+
+    /**
+     * Add a complete H6 element.
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    Heading h6(String selector, String cdata);
+  }
+
+  /** %list */
+  public interface Listing {
+
+    /**
+     * Add a UL (unordered list) element.
+     * @return a new UL element builder
+     */
+    UL ul();
+
+    /**
+     * Add a UL (unordered list) element.
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @return a new UL element builder
+     */
+    UL ul(String selector);
+
+    /**
+     * Add a OL (ordered list) element.
+     * @return a new UL element builder
+     */
+    OL ol();
+
+    /**
+     * Add a OL (ordered list) element.
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @return a new UL element builder
+     */
+    OL ol(String selector);
+  }
+
+  /** % preformatted */
+  public interface Preformatted {
+
+    /**
+     * Add a PRE (preformatted) element.
+     * @return a new PRE element builder
+     */
+    PRE pre();
+
+    /**
+     * Add a PRE (preformatted) element.
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @return a new PRE element builder
+     */
+    PRE pre(String selector);
+  }
+
+  /** %coreattrs */
+  public interface CoreAttrs {
+    /** document-wide unique id
+     * @param id the id
+     * @return the current element builder
+     */
+    CoreAttrs $id(String id);
+
+    /** space-separated list of classes
+     * @param cls the classes
+     * @return the current element builder
+     */
+    CoreAttrs $class(String cls);
+
+    /** associated style info
+     * @param style the style
+     * @return the current element builder
+     */
+    CoreAttrs $style(String style);
+
+    /** advisory title
+     * @param title the title
+     * @return the current element builder
+     */
+    CoreAttrs $title(String title);
+  }
+
+  /** %i18n */
+  public interface I18nAttrs {
+    /** language code
+     * @param lang the code
+     * @return the current element builder
+     */
+    I18nAttrs $lang(String lang);
+
+    /** direction for weak/neutral text
+     * @param dir the {@link Dir} value
+     * @return the current element builder
+     */
+    I18nAttrs $dir(Dir dir);
+  }
+
+  /** %events */
+  public interface EventsAttrs {
+
+    /** a pointer button was clicked
+     * @param onclick the script
+     * @return the current element builder
+     */
+    EventsAttrs $onclick(String onclick);
+
+    /** a pointer button was double clicked
+     * @param ondblclick the script
+     * @return the current element builder
+     */
+    EventsAttrs $ondblclick(String ondblclick);
+
+    /** a pointer button was pressed down
+     * @param onmousedown the script
+     * @return the current element builder
+     */
+    EventsAttrs $onmousedown(String onmousedown);
+
+    /** a pointer button was released
+     * @param onmouseup the script
+     * @return the current element builder
+     */
+    EventsAttrs $onmouseup(String onmouseup);
+
+    /** a pointer was moved onto
+     * @param onmouseover the script
+     * @return the current element builder
+     */
+    EventsAttrs $onmouseover(String onmouseover);
+
+    /** a pointer was moved within
+     * @param onmousemove the script
+     * @return the current element builder
+     */
+    EventsAttrs $onmousemove(String onmousemove);
+
+    /** a pointer was moved away
+     * @param onmouseout the script
+     * @return the current element builder
+     */
+    EventsAttrs $onmouseout(String onmouseout);
+
+    /** a key was pressed and released
+     * @param onkeypress the script
+     * @return the current element builder
+     */
+    EventsAttrs $onkeypress(String onkeypress);
+
+    /** a key was pressed down
+     * @param onkeydown the script
+     * @return the current element builder
+     */
+    EventsAttrs $onkeydown(String onkeydown);
+
+    /** a key was released
+     * @param onkeyup the script
+     * @return the current element builder
+     */
+    EventsAttrs $onkeyup(String onkeyup);
+  }
+
+  /** %attrs */
+  public interface Attrs extends CoreAttrs, I18nAttrs, EventsAttrs {
+  }
+
+  /** Part of %pre.exclusion */
+  public interface _FontSize extends _Child {
+    // BIG omitted cf. http://www.w3.org/TR/html5-diff/
+
+    /**
+     * Add a SMALL (small print) element
+     * @return a new SMALL element builder
+     */
+    SMALL small();
+
+    /**
+     * Add a complete small (small print) element.
+     * Shortcut of: small().__(cdata).__();
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    _FontSize small(String cdata);
+
+    /**
+     * Add a complete small (small print) element.
+     * Shortcut of: small().$id(id).$class(class).__(cdata).__();
+     * @param selector css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    _FontSize small(String selector, String cdata);
+  }
+
+  /** %fontstyle -(%pre.exclusion) */
+  public interface _FontStyle extends _Child {
+    // TT omitted
+
+    /**
+     * Add an I (italic, alt voice/mood) element.
+     * @return the new I element builder
+     */
+    I i();
+
+    /**
+     * Add a complete I (italic, alt voice/mood) element.
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    _FontStyle i(String cdata);
+
+    /**
+     * Add a complete I (italic, alt voice/mood) element.
+     * @param selector the css selector in the form of (#id)?(.class)*
+     * @param cdata the content of the element
+     * @return the current element builder
+     */
+    _FontStyle i(String selector, String cdata);
+
+    /**
+     * Add a new B (bold/important) element.
+     * @return a new B element builder
+     */
+    B b();
+
+    /**
+     * Add a complete B (bold/important) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _FontStyle b(String cdata);
+
+    /**
+     * Add a complete B (bold/important) element.
+     * @param selector the css select (#id)?(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+     _FontStyle b(String selector, String cdata);
+  }
+
+  /** %fontstyle */
+  public interface FontStyle extends _FontStyle, _FontSize {
+  }
+
+  /** %phrase */
+  public interface Phrase extends _Child {
+
+    /**
+     * Add an EM (emphasized) element.
+     * @return a new EM element builder
+     */
+    EM em();
+
+    /**
+     * Add an EM (emphasized) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase em(String cdata);
+
+    /**
+     * Add an EM (emphasized) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase em(String selector, String cdata);
+
+    /**
+     * Add a STRONG (important) element.
+     * @return a new STRONG element builder
+     */
+    STRONG strong();
+
+    /**
+     * Add a complete STRONG (important) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase strong(String cdata);
+
+    /**
+     * Add a complete STRONG (important) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase strong(String selector, String cdata);
+
+    /**
+     * Add a DFN element.
+     * @return a new DFN element builder
+     */
+    DFN dfn();
+
+    /**
+     * Add a complete DFN element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase dfn(String cdata);
+
+    /**
+     * Add a complete DFN element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase dfn(String selector, String cdata);
+
+    /**
+     * Add a CODE (code fragment) element.
+     * @return a new CODE element builder
+     */
+    CODE code();
+
+    /**
+     * Add a complete CODE element.
+     * @param cdata the code
+     * @return the current element builder
+     */
+    Phrase code(String cdata);
+
+    /**
+     * Add a complete CODE element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the code
+     * @return the current element builder
+     */
+    Phrase code(String selector, String cdata);
+
+    /**
+     * Add a SAMP (sample) element.
+     * @return a new SAMP element builder
+     */
+    SAMP samp();
+
+    /**
+     * Add a complete SAMP (sample) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase samp(String cdata);
+
+    /**
+     * Add a complete SAMP (sample) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase samp(String selector, String cdata);
+
+    /**
+     * Add a KBD (keyboard) element.
+     * @return a new KBD element builder
+     */
+    KBD kbd();
+
+    /**
+     * Add a KBD (keyboard) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase kbd(String cdata);
+
+    /**
+     * Add a KBD (keyboard) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase kbd(String selector, String cdata);
+
+    /**
+     * Add a VAR (variable) element.
+     * @return a new VAR element builder
+     */
+    VAR var();
+
+    /**
+     * Add a VAR (variable) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase var(String cdata);
+
+    /**
+     * Add a VAR (variable) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase var(String selector, String cdata);
+
+    /**
+     * Add a CITE element.
+     * @return a new CITE element builder
+     */
+    CITE cite();
+
+    /**
+     * Add a CITE element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase cite(String cdata);
+
+    /**
+     * Add a CITE element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase cite(String selector, String cdata);
+
+    /**
+     * Add an ABBR (abbreviation) element.
+     * @return a new ABBR element builder
+     */
+    ABBR abbr();
+
+    /**
+     * Add a ABBR (abbreviation) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase abbr(String cdata);
+
+    /**
+     * Add a ABBR (abbreviation) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    Phrase abbr(String selector, String cdata);
+
+    // ACRONYM omitted, use ABBR
+  }
+
+  /** Part of %pre.exclusion */
+  public interface _ImgObject extends _Object, _Child {
+
+    /**
+     * Add a IMG (image) element.
+     * @return a new IMG element builder
+     */
+    IMG img();
+
+    /**
+     * Add a IMG (image) element.
+     * @param src the source URL of the image
+     * @return the current element builder
+     */
+    _ImgObject img(String src);
+  }
+
+  /** Part of %pre.exclusion */
+  public interface _SubSup extends _Child {
+
+    /**
+     * Add a SUB (subscript) element.
+     * @return a new SUB element builder
+     */
+    SUB sub();
+
+    /**
+     * Add a complete SUB (subscript) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _SubSup sub(String cdata);
+
+    /**
+     * Add a complete SUB (subscript) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _SubSup sub(String selector, String cdata);
+
+    /**
+     * Add a SUP (superscript) element.
+     * @return a new SUP element builder
+     */
+    SUP sup();
+
+    /**
+     * Add a SUP (superscript) element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _SubSup sup(String cdata);
+
+    /**
+     * Add a SUP (superscript) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _SubSup sup(String selector, String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface _Anchor {
+
+    /**
+     * Add a A (anchor) element.
+     * @return a new A element builder
+     */
+    A a();
+
+    /**
+     * Add a A (anchor) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new A element builder
+     */
+    A a(String selector);
+
+    /** Shortcut for <code>a().$href(href).__(anchorText).__();</code>
+     * @param href the URI
+     * @param anchorText for the URI
+     * @return the current element builder
+     */
+    _Anchor a(String href, String anchorText);
+
+    /** Shortcut for <code>a(selector).$href(href).__(anchorText).__();</code>
+     * @param selector in the form of (#id)?(.class)*
+     * @param href the URI
+     * @param anchorText for the URI
+     * @return the current element builder
+     */
+    _Anchor a(String selector, String href, String anchorText);
+  }
+
+  /**
+   * INS and DEL are unusual for HTML
+   * "in that they may serve as either block-level or inline elements
+   * (but not both)".
+   * <br>cf. http://www.w3.org/TR/html4/struct/text.html#h-9.4
+   * <br>cf. http://www.w3.org/TR/html5/edits.html#edits
+   */
+  public interface _InsDel {
+
+    /**
+     * Add an INS (insert) element.
+     * @return an INS element builder
+     */
+    INS ins();
+
+    /**
+     * Add a complete INS element.
+     * @param cdata inserted data
+     * @return the current element builder
+     */
+    _InsDel ins(String cdata);
+
+    /**
+     * Add a DEL (delete) element.
+     * @return a DEL element builder
+     */
+    DEL del();
+
+    /**
+     * Add a complete DEL element.
+     * @param cdata deleted data
+     * @return the current element builder
+     */
+    _InsDel del(String cdata);
+  }
+
+  /** %special -(A|%pre.exclusion) */
+  public interface _Special extends _Script, _InsDel {
+
+    /**
+     * Add a BR (line break) element.
+     * @return a new BR element builder
+     */
+    BR br();
+
+    /**
+     * Add a BR (line break) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return the current element builder
+     */
+    _Special br(String selector);
+
+    /**
+     * Add a MAP element.
+     * @return a new MAP element builder
+     */
+    MAP map();
+
+    /**
+     * Add a MAP element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new MAP element builder
+     */
+    MAP map(String selector);
+
+    /**
+     * Add a Q (inline quotation) element.
+     * @return a q (inline quotation) element builder
+     */
+    Q q();
+
+    /**
+     * Add a complete Q element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Special q(String cdata);
+
+    /**
+     * Add a Q element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Special q(String selector, String cdata);
+
+    /**
+     * Add a SPAN element.
+     * @return a new SPAN element builder
+     */
+    SPAN span();
+
+    /**
+     * Add a SPAN element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Special span(String cdata);
+
+    /**
+     * Add a SPAN element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Special span(String selector, String cdata);
+
+    /**
+     * Add a bdo (bidirectional override) element
+     * @return a bdo element builder
+     */
+    BDO bdo();
+
+    /**
+     * Add a bdo (bidirectional override) element
+     * @param dir the direction of the text
+     * @param cdata the text
+     * @return the current element builder
+     */
+    _Special bdo(Dir dir, String cdata);
+  }
+
+  /** %special */
+  public interface Special extends _Anchor, _ImgObject, _SubSup, _Special {
+  }
+
+  /**
+   *
+   */
+  public interface _Label extends _Child {
+
+    /**
+     * Add a LABEL element.
+     * @return a new LABEL element builder
+     */
+    LABEL label();
+
+    /**
+     * Add a LABEL element.
+     * Shortcut of <code>label().$for(forId).__(cdata).__();</code>
+     * @param forId the for attribute
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Label label(String forId, String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface _FormCtrl {
+
+    /**
+     * Add a INPUT element.
+     * @return a new INPUT element builder
+     */
+    INPUT input();
+
+    /**
+     * Add a INPUT element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new INPUT element builder
+     */
+    INPUT input(String selector);
+
+    /**
+     * Add a SELECT element.
+     * @return a new SELECT element builder
+     */
+    SELECT select();
+
+    /**
+     * Add a SELECT element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new SELECT element builder
+     */
+    SELECT select(String selector);
+
+    /**
+     * Add a TEXTAREA element.
+     * @return a new TEXTAREA element builder
+     */
+    TEXTAREA textarea();
+
+    /**
+     * Add a TEXTAREA element.
+     * @param selector
+     * @return a new TEXTAREA element builder
+     */
+    TEXTAREA textarea(String selector);
+
+    /**
+     * Add a complete TEXTAREA element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _FormCtrl textarea(String selector, String cdata);
+
+    /**
+     * Add a BUTTON element.
+     * @return a new BUTTON element builder
+     */
+    BUTTON button();
+
+    /**
+     * Add a BUTTON element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new BUTTON element builder
+     */
+    BUTTON button(String selector);
+
+    /**
+     * Add a complete BUTTON element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _FormCtrl button(String selector, String cdata);
+  }
+
+  /** %formctrl */
+  public interface FormCtrl extends _Label, _FormCtrl {
+  }
+
+  /**
+   *
+   */
+  public interface _Content extends _Child {
+    /**
+     * Content of the element
+     * @param lines of content
+     * @return the current element builder
+     */
+    _Content __(Object... lines);
+  }
+
+  /**
+   *
+   */
+  public interface _RawContent extends _Child {
+    /**
+     * Raw (no need to be HTML escaped) content
+     * @param lines of content
+     * @return the current element builder
+     */
+    _RawContent _r(Object... lines);
+  }
+
+  /** #PCDATA */
+  public interface PCData extends _Content, _RawContent {
+  }
+
+  /** %inline */
+  public interface Inline extends PCData, FontStyle, Phrase, Special, FormCtrl {
+  }
+
+  /**
+   *
+   */
+  public interface I extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface B extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface SMALL extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface EM extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface STRONG extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface DFN extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface CODE extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface SAMP extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface KBD extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface VAR extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface CITE extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface ABBR extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface ACRONYM extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface SUB extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface SUP extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface SPAN extends Attrs, Inline, _Child {
+  }
+
+  /** The dir attribute is required for the BDO element */
+  public interface BDO extends CoreAttrs, I18nAttrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface BR extends CoreAttrs, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface _Form {
+
+    /**
+     * Add a FORM element.
+     * @return a new FORM element builder
+     */
+    FORM form();
+
+    /**
+     * Add a FORM element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new FORM element builder
+     */
+    FORM form(String selector);
+  }
+
+  /**
+   *
+   */
+  public interface _FieldSet {
+
+    /**
+     * Add a FIELDSET element.
+     * @return a new FIELDSET element builder
+     */
+    FIELDSET fieldset();
+
+    /**
+     * Add a FIELDSET element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new FIELDSET element builder
+     */
+    FIELDSET fieldset(String selector);
+  }
+
+  /** %block -(FORM|FIELDSET) */
+  public interface _Block extends Heading, Listing, Preformatted {
+
+    /**
+     * Add a P (paragraph) element.
+     * @return a new P element builder
+     */
+    P p();
+
+    /**
+     * Add a P (paragraph) element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new P element builder
+     */
+    P p(String selector);
+
+    /**
+     * Add a DL (description list) element.
+     * @return a new DL element builder
+     */
+    DL dl();
+
+    /**
+     * Add a DL element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new DL element builder
+     */
+    DL dl(String selector);
+
+    /**
+     * Add a DIV element.
+     * @return a new DIV element builder
+     */
+    DIV div();
+
+    /**
+     * Add a DIV element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new DIV element builder
+     */
+    DIV div(String selector);
+
+    // NOSCRIPT omitted
+    // cf. http://www.w3.org/html/wg/tracker/issues/117
+
+    /**
+     * Add a BLOCKQUOTE element.
+     * @return a new BLOCKQUOTE element builder
+     */
+    BLOCKQUOTE blockquote();
+
+    /**
+     * Alias of blockquote
+     * @return a new BLOCKQUOTE element builder
+     */
+    BLOCKQUOTE bq();
+
+    /**
+     * Add a HR (horizontal rule) element.
+     * @return a new HR element builder
+     */
+    HR hr();
+
+    /**
+     * Add a HR element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new HR element builder
+     */
+    _Block hr(String selector);
+
+    /**
+     * Add a TABLE element.
+     * @return a new TABLE element builder
+     */
+    TABLE table();
+
+    /**
+     * Add a TABLE element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new TABLE element builder
+     */
+    TABLE table(String selector);
+
+    /**
+     * Add a ADDRESS element.
+     * @return a new ADDRESS element builder
+     */
+    ADDRESS address();
+
+    /**
+     * Add a complete ADDRESS element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Block address(String cdata);
+
+    /**
+     * Embed a sub-view.
+     * @param cls the sub-view class
+     * @return the current element builder
+     */
+    _Block __(Class<? extends SubView> cls);
+  }
+
+  /** %block */
+  public interface Block extends _Block, _Form, _FieldSet {
+  }
+
+  /** %flow */
+  public interface Flow extends Block, Inline {
+  }
+
+  /**
+   *
+   */
+  public interface _Body extends Block, _Script, _InsDel {
+  }
+
+  /**
+   *
+   */
+  public interface BODY extends Attrs, _Body, _Child {
+
+    /**
+     * The document has been loaded.
+     * @param script to invoke
+     * @return the current element builder
+     */
+    BODY $onload(String script);
+
+    /**
+     * The document has been removed
+     * @param script to invoke
+     * @return the current element builder
+     */
+    BODY $onunload(String script);
+  }
+
+  /**
+   *
+   */
+  public interface ADDRESS extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface DIV extends Attrs, Flow, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface A extends Attrs, _Child, /* %inline -(A) */
+                             PCData, FontStyle, Phrase, _ImgObject, _Special,
+                             _SubSup, FormCtrl {
+    // $charset omitted.
+
+    /** advisory content type
+     * @param cdata the content-type
+     * @return the current element builder
+     */
+    A $type(String cdata);
+
+    // $name omitted. use id instead.
+    /** URI for linked resource
+     * @param uri the URI
+     * @return the current element builder
+     */
+    A $href(String uri);
+
+    /** language code
+     * @param cdata the code
+     * @return the current element builder
+     */
+    A $hreflang(String cdata);
+
+    /** forward link types
+     * @param linkTypes the types
+     * @return the current element builder
+     */
+    A $rel(EnumSet<LinkType> linkTypes);
+
+    /**
+     * forward link types
+     * @param linkTypes space-separated list of link types
+     * @return the current element builder.
+     */
+    A $rel(String linkTypes);
+
+    // $rev omitted. Instead of rev="made", use rel="author"
+
+    /** accessibility key character
+     * @param cdata the key
+     * @return the current element builder
+     */
+    A $accesskey(String cdata);
+
+    // $shape and coords omitted. use area instead of a for image maps.
+    /** position in tabbing order
+     * @param index the index
+     * @return the current element builder
+     */
+    A $tabindex(int index);
+
+    /** the element got the focus
+     * @param script to invoke
+     * @return the current element builder
+     */
+    A $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script to invoke
+     * @return the current element builder
+     */
+    A $onblur(String script);
+  }
+
+  /**
+   *
+   */
+  public interface MAP extends Attrs, Block, _Child {
+
+    /**
+     * Add a AREA element.
+     * @return a new AREA element builder
+     */
+    AREA area();
+
+    /**
+     * Add a AREA element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new AREA element builder
+     */
+    AREA area(String selector);
+
+    /** for reference by usemap
+     * @param name of the map
+     * @return the current element builder
+     */
+    MAP $name(String name);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface AREA extends Attrs, _Child {
+
+    /** controls interpretation of coords
+     * @param shape of the area
+     * @return the current element builder
+     */
+    AREA $shape(Shape shape);
+
+    /** comma-separated list of lengths
+     * @param cdata coords of the area
+     * @return the current element builder
+     */
+    AREA $coords(String cdata);
+
+    /** URI for linked resource
+     * @param uri the URI
+     * @return the current element builder
+     */
+    AREA $href(String uri);
+
+    // $nohref omitted./
+    /** short description
+     * @param desc the description
+     * @return the current element builder
+     */
+    AREA $alt(String desc);
+
+    /** position in tabbing order
+     * @param index of the order
+     * @return the current element builder
+     */
+    AREA $tabindex(int index);
+
+    /** accessibility key character
+     * @param cdata the key
+     * @return the current element builder
+     */
+    AREA $accesskey(String cdata);
+
+    /** the element got the focus
+     * @param script to invoke
+     * @return the current element builder
+     */
+    AREA $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script to invoke
+     * @return the current element builder
+     */
+    AREA $onblur(String script);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface LINK extends Attrs, _Child {
+    // $charset omitted
+    /** URI for linked resource
+     * @param uri the URI
+     * @return the current element builder
+     */
+    LINK $href(String uri);
+
+    /** language code
+     * @param cdata the code
+     * @return the current element builder
+     */
+    LINK $hreflang(String cdata);
+
+    /** advisory content type
+     * @param cdata the type
+     * @return the current element builder
+     */
+    LINK $type(String cdata);
+
+    /** forward link types
+     * @param linkTypes the types
+     * @return the current element builder
+     */
+    LINK $rel(EnumSet<LinkType> linkTypes);
+
+    /**
+     * forward link types.
+     * @param linkTypes space-separated link types
+     * @return the current element builder
+     */
+    LINK $rel(String linkTypes);
+
+    // $rev omitted. Instead of rev="made", use rel="author"
+
+    /** for rendering on these media
+     * @param mediaTypes the media types
+     * @return the current element builder
+     */
+    LINK $media(EnumSet<Media> mediaTypes);
+
+    /**
+     * for rendering on these media.
+     * @param mediaTypes comma-separated list of media
+     * @return the current element builder
+     */
+    LINK $media(String mediaTypes);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface IMG extends Attrs, _Child {
+
+    /** URI of image to embed
+     * @param uri the URI
+     * @return the current element builder
+     */
+    IMG $src(String uri);
+
+    /** short description
+     * @param desc the description
+     * @return the current element builder
+     */
+    IMG $alt(String desc);
+
+    // $longdesc omitted. use <a...><img..></a> instead
+    // $name omitted. use id instead.
+
+    /** override height
+     * @param pixels the height
+     * @return the current element builder
+     */
+    IMG $height(int pixels);
+
+    /**
+     * override height
+     * @param cdata the height (can use %, * etc.)
+     * @return the current element builder
+     */
+    IMG $height(String cdata);
+
+    /** override width
+     * @param pixels the width
+     * @return the current element builder
+     */
+    IMG $width(int pixels);
+
+    /**
+     * override width
+     * @param cdata the width (can use %, * etc.)
+     * @return the current element builder
+     */
+    IMG $width(String cdata);
+
+    /** use client-side image map
+     * @param uri the URI
+     * @return the current element builder
+     */
+    IMG $usemap(String uri);
+
+    /** use server-side image map
+     * @return the current element builder
+     */
+    IMG $ismap();
+  }
+
+  /**
+   *
+   */
+  public interface _Param extends _Child {
+
+    /**
+     * Add a PARAM (parameter) element.
+     * @return a new PARAM element builder
+     */
+    PARAM param();
+
+    /**
+     * Add a PARAM element.
+     * Shortcut of <code>param().$name(name).$value(value).__();</code>
+     * @param name of the value
+     * @param value the value
+     * @return the current element builder
+     */
+    _Param param(String name, String value);
+  }
+
+  /**
+   *
+   */
+  public interface OBJECT extends Attrs, _Param, Flow, _Child {
+    // $declare omitted. repeat element completely
+
+    // $archive, classid, codebase, codetype ommited. use data and type
+
+    /** reference to object's data
+     * @param uri the URI
+     * @return the current element builder
+     */
+    OBJECT $data(String uri);
+
+    /** content type for data
+     * @param contentType the type
+     * @return the current element builder
+     */
+    OBJECT $type(String contentType);
+
+    // $standby omitted. fix the resource instead.
+
+    /** override height
+     * @param pixels the height
+     * @return the current element builder
+     */
+    OBJECT $height(int pixels);
+
+    /**
+     * override height
+     * @param length the height (can use %, *)
+     * @return the current element builder
+     */
+    OBJECT $height(String length);
+
+    /** override width
+     * @param pixels the width
+     * @return the current element builder
+     */
+    OBJECT $width(int pixels);
+
+    /**
+     * override width
+     * @param length the height (can use %, *)
+     * @return the current element builder
+     */
+    OBJECT $width(String length);
+
+    /** use client-side image map
+     * @param uri the URI/name of the map
+     * @return the current element builder
+     */
+    OBJECT $usemap(String uri);
+
+    /** submit as part of form
+     * @param cdata the name of the object
+     * @return the current element builder
+     */
+    OBJECT $name(String cdata);
+
+    /** position in tabbing order
+     * @param index of the order
+     * @return the current element builder
+     */
+    OBJECT $tabindex(int index);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface PARAM {
+
+    /** document-wide unique id
+     * @param cdata the id
+     * @return the current element builder
+     */
+    PARAM $id(String cdata);
+
+    /** property name. Required.
+     * @param cdata the name
+     * @return the current element builder
+     */
+    PARAM $name(String cdata);
+
+    /** property value
+     * @param cdata the value
+     * @return the current element builder
+     */
+    PARAM $value(String cdata);
+
+    // $type and valuetype omitted
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface HR extends Attrs, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface P extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H1 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H2 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H3 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H4 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H5 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface H6 extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface PRE extends Attrs, _Child, /* (%inline;)* -(%pre.exclusion) */
+                               PCData, _FontStyle, Phrase, _Anchor, _Special,
+                               FormCtrl {
+  }
+
+  /**
+   *
+   */
+  public interface Q extends Attrs, Inline, _Child {
+
+    /** URI for source document or msg
+     * @param uri the URI
+     * @return the current element builder
+     */
+    Q $cite(String uri);
+  }
+
+  /**
+   *
+   */
+  public interface BLOCKQUOTE extends Attrs, Block, _Script, _Child {
+
+    /** URI for source document or msg
+     * @param uri the URI
+     * @return the current element builder
+     */
+    BLOCKQUOTE $cite(String uri);
+  }
+
+  /**
+   * @see _InsDel INS/DEL quirks.
+   */
+  public interface INS extends Attrs, Flow, _Child {
+    /** info on reason for change
+     * @param uri
+     * @return the current element builder
+     */
+    INS $cite(String uri);
+
+    /** date and time of change
+     * @param datetime
+     * @return the current element builder
+     */
+    INS $datetime(String datetime);
+  }
+
+  /**
+   * @see _InsDel INS/DEL quirks.
+   */
+  public interface DEL extends Attrs, Flow, _Child {
+    /** info on reason for change
+     * @param uri the info URI
+     * @return the current element builder
+     */
+    DEL $cite(String uri);
+
+    /** date and time of change
+     * @param datetime the time
+     * @return the current element builder
+     */
+    DEL $datetime(String datetime);
+  }
+
+  /**
+   *
+   */
+  public interface _Dl extends _Child {
+
+    /**
+     * Add a DT (term of the item) element.
+     * @return a new DT element builder
+     */
+    DT dt();
+
+    /**
+     * Add a complete DT element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Dl dt(String cdata);
+
+    /**
+     * Add a DD (definition/description) element.
+     * @return a new DD element builder
+     */
+    DD dd();
+
+    /**
+     * Add a complete DD element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Dl dd(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface DL extends Attrs, _Dl, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface DT extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface DD extends Attrs, Flow, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface _Li extends _Child {
+
+    /**
+     * Add a LI (list item) element.
+     * @return a new LI element builder
+     */
+    LI li();
+
+    /**
+     * Add a LI element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Li li(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface OL extends Attrs, _Li, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface UL extends Attrs, _Li, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface LI extends Attrs, Flow, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface FORM extends Attrs, _Child, /* (%block;|SCRIPT)+ -(FORM) */
+                                _Script, _Block, _FieldSet {
+    /** server-side form handler
+     * @param uri
+     * @return the current element builder
+     */
+    FORM $action(String uri);
+
+    /** HTTP method used to submit the form
+     * @param method
+     * @return the current element builder
+     */
+    FORM $method(Method method);
+
+    /**
+     * contentype for "POST" method.
+     * The default is "application/x-www-form-urlencoded".
+     * Use "multipart/form-data" for input type=file
+     * @param enctype
+     * @return the current element builder
+     */
+    FORM $enctype(String enctype);
+
+    /** list of MIME types for file upload
+     * @param cdata
+     * @return the current element builder
+     */
+    FORM $accept(String cdata);
+
+    /** name of form for scripting
+     * @param cdata
+     * @return the current element builder
+     */
+    FORM $name(String cdata);
+
+    /** the form was submitted
+     * @param script
+     * @return the current element builder
+     */
+    FORM $onsubmit(String script);
+
+    /** the form was reset
+     * @param script
+     * @return the current element builder
+     */
+    FORM $onreset(String script);
+
+    /** (space and/or comma separated) list of supported charsets
+     * @param cdata
+     * @return the current element builder
+     */
+    FORM $accept_charset(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface LABEL extends Attrs, _Child, /* (%inline;)* -(LABEL) */
+                                 PCData, FontStyle, Phrase, Special, _FormCtrl {
+    /** matches field ID value
+     * @param cdata
+     * @return the current element builder
+     */
+    LABEL $for(String cdata);
+
+    /** accessibility key character
+     * @param cdata
+     * @return the current element builder
+     */
+    LABEL $accesskey(String cdata);
+
+    /** the element got the focus
+     * @param script
+     * @return the current element builder
+     */
+    LABEL $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script
+     * @return the current element builder
+     */
+    LABEL $onblur(String script);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface INPUT extends Attrs, _Child {
+    /** what kind of widget is needed. default is "text".
+     * @param inputType
+     * @return the current element builder
+     */
+    INPUT $type(InputType inputType);
+
+    /** submit as part of form
+     * @param cdata
+     * @return the current element builder
+     */
+    INPUT $name(String cdata);
+
+    /** Specify for radio buttons and checkboxes
+     * @param cdata
+     * @return the current element builder
+     */
+    INPUT $value(String cdata);
+
+    /** for radio buttons and check boxes
+     * @return the current element builder
+     */
+    INPUT $checked();
+
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    INPUT $disabled();
+
+    /** for text and passwd
+     * @return the current element builder
+     */
+    INPUT $readonly();
+
+    /** specific to each type of field
+     * @param cdata
+     * @return the current element builder
+     */
+    INPUT $size(String cdata);
+
+    /** max chars for text fields
+     * @param length
+     * @return the current element builder
+     */
+    INPUT $maxlength(int length);
+
+    /** for fields with images
+     * @param uri
+     * @return the current element builder
+     */
+    INPUT $src(String uri);
+
+    /** short description
+     * @param cdata
+     * @return the current element builder
+     */
+    INPUT $alt(String cdata);
+
+    // $usemap omitted. use img instead of input for image maps.
+    /** use server-side image map
+     * @return the current element builder
+     */
+    INPUT $ismap();
+
+    /** position in tabbing order
+     * @param index
+     * @return the current element builder
+     */
+    INPUT $tabindex(int index);
+
+    /** accessibility key character
+     * @param cdata
+     * @return the current element builder
+     */
+    INPUT $accesskey(String cdata);
+
+    /** the element got the focus
+     * @param script
+     * @return the current element builder
+     */
+    INPUT $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script
+     * @return the current element builder
+     */
+    INPUT $onblur(String script);
+
+    /** some text was selected
+     * @param script
+     * @return the current element builder
+     */
+    INPUT $onselect(String script);
+
+    /** the element value was changed
+     * @param script
+     * @return the current element builder
+     */
+    INPUT $onchange(String script);
+
+    /** list of MIME types for file upload (csv)
+     * @param contentTypes
+     * @return the current element builder
+     */
+    INPUT $accept(String contentTypes);
+  }
+
+  /**
+   *
+   */
+  public interface _Option extends _Child {
+    /**
+     * Add a OPTION element.
+     * @return a new OPTION element builder
+     */
+    OPTION option();
+
+    /**
+     * Add a complete OPTION element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Option option(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface SELECT extends Attrs, _Option, _Child {
+    /**
+     * Add a OPTGROUP element.
+     * @return a new OPTGROUP element builder
+     */
+    OPTGROUP optgroup();
+
+    /** field name
+     * @param cdata
+     * @return the current element builder
+     */
+    SELECT $name(String cdata);
+
+    /** rows visible
+     * @param rows
+     * @return the current element builder
+     */
+    SELECT $size(int rows);
+
+    /** default is single selection
+     * @return the current element builder
+     */
+    SELECT $multiple();
+
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    SELECT $disabled();
+
+    /** position in tabbing order
+     * @param index
+     * @return the current element builder
+     */
+    SELECT $tabindex(int index);
+
+    /** the element got the focus
+     * @param script
+     * @return the current element builder
+     */
+    SELECT $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script
+     * @return the current element builder
+     */
+    SELECT $onblur(String script);
+
+    /** the element value was changed
+     * @param script
+     * @return the current element builder
+     */
+    SELECT $onchange(String script);
+  }
+
+  /**
+   *
+   */
+  public interface OPTGROUP extends Attrs, _Option, _Child {
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    OPTGROUP $disabled();
+
+    /** for use in hierarchical menus
+     * @param cdata
+     * @return the current element builder
+     */
+    OPTGROUP $label(String cdata);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface OPTION extends Attrs, PCData, _Child {
+    /** currently selected option
+     * @return the current element builder
+     */
+    OPTION $selected();
+
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    OPTION $disabled();
+
+    /** for use in hierarchical menus
+     * @param cdata
+     * @return the current element builder
+     */
+    OPTION $label(String cdata);
+
+    /** defaults to element content
+     * @param cdata
+     * @return the current element builder
+     */
+    OPTION $value(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface TEXTAREA extends Attrs, PCData, _Child {
+    /** variable name for the text
+     * @param cdata
+     * @return the current element builder
+     */
+    TEXTAREA $name(String cdata);
+
+    /** visible rows
+     * @param rows
+     * @return the current element builder
+     */
+    TEXTAREA $rows(int rows);
+
+    /** visible columns
+     * @param cols
+     * @return the current element builder
+     */
+    TEXTAREA $cols(int cols);
+
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    TEXTAREA $disabled();
+
+    /** text is readonly
+     * @return the current element builder
+     */
+    TEXTAREA $readonly();
+
+    /** position in tabbing order
+     * @param index
+     * @return the current element builder
+     */
+    TEXTAREA $tabindex(int index);
+
+    /** accessibility key character
+     * @param cdata
+     * @return the current element builder
+     */
+    TEXTAREA $accesskey(String cdata);
+
+    /** the element got the focus
+     * @param script
+     * @return the current element builder
+     */
+    TEXTAREA $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script
+     * @return the current element builder
+     */
+    TEXTAREA $onblur(String script);
+
+    /** some text was selected
+     * @param script
+     * @return the current element builder
+     */
+    TEXTAREA $onselect(String script);
+
+    /** the element value was changed
+     * @param script
+     * @return the current element builder
+     */
+    TEXTAREA $onchange(String script);
+  }
+
+  /**
+   *
+   */
+  public interface _Legend extends _Child {
+    /**
+     * Add a LEGEND element.
+     * @return a new LEGEND element builder
+     */
+    LEGEND legend();
+
+    /**
+     * Add a LEGEND element.
+     * @param cdata
+     * @return the current element builder
+     */
+    _Legend legend(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface FIELDSET extends Attrs, _Legend, PCData, Flow, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface LEGEND extends Attrs, Inline, _Child {
+    /** accessibility key character
+     * @param cdata
+     * @return the current element builder
+     */
+    LEGEND $accesskey(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface BUTTON extends /* (%flow;)* -(A|%formctrl|FORM|FIELDSET) */
+      _Block, PCData, FontStyle, Phrase, _Special, _ImgObject, _SubSup, Attrs {
+    /** name of the value
+     * @param cdata
+     * @return the current element builder
+     */
+    BUTTON $name(String cdata);
+
+    /** sent to server when submitted
+     * @param cdata
+     * @return the current element builder
+     */
+    BUTTON $value(String cdata);
+
+    /** for use as form button
+     * @param type
+     * @return the current element builder
+     */
+    BUTTON $type(ButtonType type);
+
+    /** unavailable in this context
+     * @return the current element builder
+     */
+    BUTTON $disabled();
+
+    /** position in tabbing order
+     * @param index
+     * @return the current element builder
+     */
+    BUTTON $tabindex(int index);
+
+    /** accessibility key character
+     * @param cdata
+     * @return the current element builder
+     */
+    BUTTON $accesskey(String cdata);
+
+    /** the element got the focus
+     * @param script
+     * @return the current element builder
+     */
+    BUTTON $onfocus(String script);
+
+    /** the element lost the focus
+     * @param script
+     * @return the current element builder
+     */
+    BUTTON $onblur(String script);
+  }
+
+  /**
+   *
+   */
+  public interface _TableRow {
+    /**
+     * Add a TR (table row) element.
+     * @return a new TR element builder
+     */
+    TR tr();
+
+    /**
+     * Add a TR element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new TR element builder
+     */
+    TR tr(String selector);
+  }
+
+  /**
+   *
+   */
+  public interface _TableCol extends _Child {
+    /**
+     * Add a COL element.
+     * @return a new COL element builder
+     */
+    COL col();
+
+    /**
+     * Add a COL element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return the current element builder
+     */
+    _TableCol col(String selector);
+  }
+
+  /**
+   *
+   */
+  public interface _Table extends _TableRow, _TableCol {
+    /**
+     * Add a CAPTION element.
+     * @return a new CAPTION element builder
+     */
+    CAPTION caption();
+
+    /**
+     * Add a CAPTION element.
+     * @param cdata
+     * @return the current element builder
+     */
+    _Table caption(String cdata);
+
+    /**
+     * Add a COLGROPU element.
+     * @return a new COLGROUP element builder
+     */
+    COLGROUP colgroup();
+
+    /**
+     * Add a THEAD element.
+     * @return a new THEAD element builder
+     */
+    THEAD thead();
+
+    /**
+     * Add a THEAD element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new THEAD element builder
+     */
+    THEAD thead(String selector);
+
+    /**
+     * Add a TFOOT element.
+     * @return a new TFOOT element builder
+     */
+    TFOOT tfoot();
+
+    /**
+     * Add a TFOOT element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new TFOOT element builder
+     */
+    TFOOT tfoot(String selector);
+
+    /**
+     * Add a tbody (table body) element.
+     * Must be after thead/tfoot and no tr at the same level.
+     * @return a new tbody element builder
+     */
+    TBODY tbody();
+
+    /**
+     * Add a TBODY element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new TBODY element builder
+     */
+    TBODY tbody(String selector);
+
+    // $summary, width, border, frame, rules, cellpadding, cellspacing omitted
+    // use css instead
+  }
+  /**
+   * TBODY should be used after THEAD/TFOOT, iff there're no TABLE.TR elements.
+   */
+  public interface TABLE extends Attrs, _Table, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface CAPTION extends Attrs, Inline, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface THEAD extends Attrs, _TableRow, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface TFOOT extends Attrs, _TableRow, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface TBODY extends Attrs, _TableRow, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface COLGROUP extends Attrs, _TableCol, _Child {
+    /** default number of columns in group. default: 1
+     * @param cols
+     * @return the current element builder
+     */
+    COLGROUP $span(int cols);
+
+    // $width omitted. use css instead.
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface COL extends Attrs, _Child {
+    /** COL attributes affect N columns. default: 1
+     * @param cols
+     * @return the current element builder
+     */
+    COL $span(int cols);
+    // $width omitted. use css instead.
+  }
+
+  /**
+   *
+   */
+  public interface _Tr extends _Child {
+    /**
+     * Add a TH element.
+     * @return a new TH element builder
+     */
+    TH th();
+
+    /**
+     * Add a complete TH element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Tr th(String cdata);
+
+    /**
+     * Add a TH element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Tr th(String selector, String cdata);
+
+    /**
+     * Add a TD element.
+     * @return a new TD element builder
+     */
+    TD td();
+
+    /**
+     * Add a TD element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Tr td(String cdata);
+
+    /**
+     * Add a TD element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Tr td(String selector, String cdata);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface TR extends Attrs, _Tr, _Child {
+  }
+
+  /**
+   *
+   */
+  public interface _Cell extends Attrs, Flow, _Child {
+    // $abbr omited. begin cell text with terse text instead.
+    // use $title for elaberation, when appropriate.
+    // $axis omitted. use scope.
+    /** space-separated list of id's for header cells
+     * @param cdata
+     * @return the current element builder
+     */
+    _Cell $headers(String cdata);
+
+    /** scope covered by header cells
+     * @param scope
+     * @return the current element builder
+     */
+    _Cell $scope(Scope scope);
+
+    /** number of rows spanned by cell. default: 1
+     * @param rows
+     * @return the current element builder
+     */
+    _Cell $rowspan(int rows);
+
+    /** number of cols spanned by cell. default: 1
+     * @param cols
+     * @return the current element builder
+     */
+    _Cell $colspan(int cols);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface TH extends _Cell {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface TD extends _Cell {
+  }
+
+  /**
+   *
+   */
+  public interface _Head extends HeadMisc {
+    /**
+     * Add a TITLE element.
+     * @return a new TITLE element builder
+     */
+    TITLE title();
+
+    /**
+     * Add a TITLE element.
+     * @param cdata the content
+     * @return the current element builder
+     */
+    _Head title(String cdata);
+
+    /**
+     * Add a BASE element.
+     * @return a new BASE element builder
+     */
+    BASE base();
+
+    /**
+     * Add a complete BASE element.
+     * @param uri
+     * @return the current element builder
+     */
+    _Head base(String uri);
+  }
+
+  /**
+   *
+   */
+  public interface HEAD extends I18nAttrs, _Head, _Child {
+    // $profile omitted
+  }
+
+  /**
+   *
+   */
+  public interface TITLE extends I18nAttrs, PCData, _Child {
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface BASE extends _Child {
+    /** URI that acts as base URI
+     * @param uri
+     * @return the current element builder
+     */
+    BASE $href(String uri);
+  }
+
+  /**
+   *
+   */
+  @Element(endTag=false)
+  public interface META extends I18nAttrs, _Child {
+    /** HTTP response header name
+     * @param header
+     * @return the current element builder
+     */
+    META $http_equiv(String header);
+
+    /** metainformation name
+     * @param name
+     * @return the current element builder
+     */
+    META $name(String name);
+
+    /** associated information
+     * @param cdata
+     * @return the current element builder
+     */
+    META $content(String cdata);
+
+    // $scheme omitted
+  }
+
+  /**
+   *
+   */
+  public interface STYLE extends I18nAttrs, _Content, _Child {
+    /** content type of style language
+     * @param cdata
+     * @return the current element builder
+     */
+    STYLE $type(String cdata);
+
+    /** designed for use with these media
+     * @param media
+     * @return the current element builder
+     */
+    STYLE $media(EnumSet<Media> media);
+
+    /** advisory title
+     * @param cdata
+     * @return the current element builder
+     */
+    STYLE $title(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface SCRIPT extends _Content, _Child {
+    /** char encoding of linked resource
+     * @param cdata
+     * @return the current element builder
+     */
+    SCRIPT $charset(String cdata);
+
+    /** content type of script language
+     * @param cdata
+     * @return the current element builder
+     */
+    SCRIPT $type(String cdata);
+
+    /** URI for an external script
+     * @param cdata
+     * @return the current element builder
+     */
+    SCRIPT $src(String cdata);
+
+    /** UA may defer execution of script
+     * @param cdata
+     * @return the current element builder
+     */
+    SCRIPT $defer(String cdata);
+  }
+
+  /**
+   *
+   */
+  public interface _Html extends _Head, _Body, __ {
+    /**
+     * Add a HEAD element.
+     * @return a new HEAD element builder
+     */
+    HEAD head();
+
+    /**
+     * Add a BODY element.
+     * @return a new BODY element builder
+     */
+    BODY body();
+
+    /**
+     * Add a BODY element.
+     * @param selector the css selector in the form of (#id)*(.class)*
+     * @return a new BODY element builder
+     */
+    BODY body(String selector);
+  }
+
+  // There is only one HEAD and BODY, in that order.
+  /**
+   * The root element
+   */
+  public interface HTML extends I18nAttrs, _Html {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java
new file mode 100644
index 0000000..64a8447
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/hamlet2/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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 package is the replacement for org.apache.hadoop.yarn.webapp.hamlet.
+ * The old package is using _ as a one-character identifier,
+ * which is banned from JDK9.
+ */
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+package org.apache.hadoop.yarn.webapp.hamlet2;
+import org.apache.hadoop.classification.InterfaceAudience;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
index 1da6e23..0c7e09e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
@@ -44,10 +44,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
 import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.PRE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.PRE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -84,12 +83,12 @@ public class AggregatedLogsBlock extends HtmlBlock {
     if (!conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
         YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)) {
       html.h1()
-          ._("Aggregation is not enabled. Try the nodemanager at " + nodeId)
-          ._();
+          .__("Aggregation is not enabled. Try the nodemanager at " + nodeId)
+          .__();
       if(nmApplicationLogUrl != null) {
         html.h1()
-            ._("Or see application log at " + nmApplicationLogUrl)
-            ._();
+            .__("Or see application log at " + nmApplicationLogUrl)
+            .__();
       }
       return;
     }
@@ -110,18 +109,18 @@ public class AggregatedLogsBlock extends HtmlBlock {
             .listStatus(remoteAppDir);
     } catch (FileNotFoundException fnf) {
       html.h1()
-          ._("Logs not available for " + logEntity
+          .__("Logs not available for " + logEntity
               + ". Aggregation may not be complete, "
-              + "Check back later or try the nodemanager at " + nodeId)._();
+              + "Check back later or try the nodemanager at " + nodeId).__();
       if(nmApplicationLogUrl != null)  {
         html.h1()
-            ._("Or see application log at " + nmApplicationLogUrl)
-            ._();
+            .__("Or see application log at " + nmApplicationLogUrl)
+            .__();
       }
       return;
     } catch (Exception ex) {
       html.h1()
-          ._("Error getting logs at " + nodeId)._();
+          .__("Error getting logs at " + nodeId).__();
       return;
     }
 
@@ -168,9 +167,9 @@ public class AggregatedLogsBlock extends HtmlBlock {
           if (callerUGI != null && !aclsManager.checkAccess(callerUGI,
               ApplicationAccessType.VIEW_APP, owner, applicationId)) {
             html.h1()
-                ._("User [" + remoteUser
+                .__("User [" + remoteUser
                     + "] is not authorized to view the logs for " + logEntity
-                    + " in log file [" + thisNodeFile.getPath().getName() + "]")._();
+                    + " in log file [" + thisNodeFile.getPath().getName() + "]").__();
             LOG.error("User [" + remoteUser
               + "] is not authorized to view the logs for " + logEntity);
             continue;
@@ -188,8 +187,9 @@ public class AggregatedLogsBlock extends HtmlBlock {
           LOG.error("Error getting logs for " + logEntity, ex);
           continue;
         } finally {
-          if (reader != null)
+          if (reader != null) {
             reader.close();
+          }
         }
       }
       if (!foundLog) {
@@ -201,7 +201,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
         }
       }
     } catch (IOException e) {
-      html.h1()._("Error getting logs for " + logEntity)._();
+      html.h1().__("Error getting logs for " + logEntity).__();
       LOG.error("Error getting logs for " + logEntity, e);
     }
   }
@@ -219,12 +219,12 @@ public class AggregatedLogsBlock extends HtmlBlock {
           || desiredLogType.equals(logType)) {
         long logLength = logReader.getCurrentLogLength();
         if (foundLog) {
-          html.pre()._("\n\n")._();
+          html.pre().__("\n\n").__();
         }
 
-        html.p()._("Log Type: " + logType)._();
-        html.p()._("Log Upload Time: " + Times.format(logUpLoadTime))._();
-        html.p()._("Log Length: " + Long.toString(logLength))._();
+        html.p().__("Log Type: " + logType).__();
+        html.p().__("Log Upload Time: " + Times.format(logUpLoadTime)).__();
+        html.p().__("Log Length: " + Long.toString(logLength)).__();
 
         long start = logLimits.start < 0
             ? logLength + logLimits.start : logLimits.start;
@@ -238,12 +238,12 @@ public class AggregatedLogsBlock extends HtmlBlock {
 
         long toRead = end - start;
         if (toRead < logLength) {
-            html.p()._("Showing " + toRead + " bytes of " + logLength
+            html.p().__("Showing " + toRead + " bytes of " + logLength
                 + " total. Click ")
                 .a(url("logs", $(NM_NODENAME), $(CONTAINER_ID),
                     $(ENTITY_STRING), $(APP_OWNER),
                     logType, "?start=0"), "here").
-                    _(" for the full log.")._();
+                __(" for the full log.").__();
         }
 
         long totalSkipped = 0;
@@ -267,12 +267,12 @@ public class AggregatedLogsBlock extends HtmlBlock {
 
         while (toRead > 0
             && (len = logReader.read(cbuf, 0, currentToRead)) > 0) {
-          pre._(new String(cbuf, 0, len));
+          pre.__(new String(cbuf, 0, len));
           toRead = toRead - len;
           currentToRead = toRead > bufferSize ? bufferSize : (int) toRead;
         }
 
-        pre._();
+        pre.__();
         foundLog = true;
       }
 
@@ -285,7 +285,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
   private ContainerId verifyAndGetContainerId(Block html) {
     String containerIdStr = $(CONTAINER_ID);
     if (containerIdStr == null || containerIdStr.isEmpty()) {
-      html.h1()._("Cannot get container logs without a ContainerId")._();
+      html.h1().__("Cannot get container logs without a ContainerId").__();
       return null;
     }
     ContainerId containerId = null;
@@ -293,8 +293,8 @@ public class AggregatedLogsBlock extends HtmlBlock {
       containerId = ContainerId.fromString(containerIdStr);
     } catch (IllegalArgumentException e) {
       html.h1()
-          ._("Cannot get container logs for invalid containerId: "
-              + containerIdStr)._();
+          .__("Cannot get container logs for invalid containerId: "
+              + containerIdStr).__();
       return null;
     }
     return containerId;
@@ -303,15 +303,15 @@ public class AggregatedLogsBlock extends HtmlBlock {
   private NodeId verifyAndGetNodeId(Block html) {
     String nodeIdStr = $(NM_NODENAME);
     if (nodeIdStr == null || nodeIdStr.isEmpty()) {
-      html.h1()._("Cannot get container logs without a NodeId")._();
+      html.h1().__("Cannot get container logs without a NodeId").__();
       return null;
     }
     NodeId nodeId = null;
     try {
       nodeId = NodeId.fromString(nodeIdStr);
     } catch (IllegalArgumentException e) {
-      html.h1()._("Cannot get container logs. Invalid nodeId: " + nodeIdStr)
-          ._();
+      html.h1().__("Cannot get container logs. Invalid nodeId: " + nodeIdStr)
+          .__();
       return null;
     }
     return nodeId;
@@ -320,7 +320,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
   private String verifyAndGetAppOwner(Block html) {
     String appOwner = $(APP_OWNER);
     if (appOwner == null || appOwner.isEmpty()) {
-      html.h1()._("Cannot get container logs without an app owner")._();
+      html.h1().__("Cannot get container logs without an app owner").__();
     }
     return appOwner;
   }
@@ -341,7 +341,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
         start = Long.parseLong(startStr);
       } catch (NumberFormatException e) {
         isValid = false;
-        html.h1()._("Invalid log start value: " + startStr)._();
+        html.h1().__("Invalid log start value: " + startStr).__();
       }
     }
 
@@ -351,7 +351,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
         end = Long.parseLong(endStr);
       } catch (NumberFormatException e) {
         isValid = false;
-        html.h1()._("Invalid log end value: " + endStr)._();
+        html.h1().__("Invalid log end value: " + endStr).__();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java
index fe83eaa..a6e3a05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsNavBlock.java
@@ -28,7 +28,7 @@ public class AggregatedLogsNavBlock extends HtmlBlock {
   protected void render(Block html) {
     html
       .div("#nav")
-        .h3()._("Logs")._() // 
-      ._();
+        .h3().__("Logs").__()
+      .__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java
index 773738f..f097b0d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsPage.java
@@ -37,7 +37,7 @@ public class AggregatedLogsPage extends TwoColumnLayout {
    * @see org.apache.hadoop.yarn.server.nodemanager.webapp.NMView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     String logEntity = $(ENTITY_STRING);
     if (logEntity == null || logEntity.isEmpty()) {
       logEntity = $(CONTAINER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/ErrorPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/ErrorPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/ErrorPage.java
index 68e09ad..fabb5c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/ErrorPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/ErrorPage.java
@@ -30,24 +30,24 @@ import org.apache.hadoop.classification.InterfaceAudience;
 public class ErrorPage extends HtmlPage {
 
   @Override
-  protected void render(Page.HTML<_> html) {
+  protected void render(Page.HTML<__> html) {
     set(JQueryUI.ACCORDION_ID, "msg");
     String title = "Sorry, got error "+ status();
     html.
       title(title).
-      link(root_url("static","yarn.css")).
-      _(JQueryUI.class). // an embedded sub-view
+      link(root_url("static", "yarn.css")).
+        __(JQueryUI.class). // an embedded sub-view
       style("#msg { margin: 1em auto; width: 88%; }",
             "#msg h1 { padding: 0.2em 1.5em; font: bold 1.3em serif; }").
       div("#msg").
         h1(title).
         div().
-          _("Please consult").
+        __("Please consult").
           a("http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html",
-            "RFC 2616")._(" for meanings of the error code.")._().
+            "RFC 2616").__(" for meanings of the error code.").__().
         h1("Error Details").
         pre().
-          _(errorDetails())._()._()._();
+        __(errorDetails()).__().__().__();
   }
 
   protected String errorDetails() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/FooterBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/FooterBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/FooterBlock.java
index ba85ac6..e4d1f2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/FooterBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/FooterBlock.java
@@ -25,6 +25,6 @@ public class FooterBlock extends HtmlBlock {
 
   @Override protected void render(Block html) {
     html.
-      div("#footer.ui-widget")._();
+      div("#footer.ui-widget").__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java
index 03f0fb1..3a0f35a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java
@@ -31,9 +31,9 @@ public class HeaderBlock extends HtmlBlock {
     html.
       div("#header.ui-widget").
         div("#user").
-          _(loggedIn)._().
+        __(loggedIn).__().
         div("#logo").
-          img("/static/hadoop-st.png")._().
-        h1($(TITLE))._();
+          img("/static/hadoop-st.png").__().
+        h1($(TITLE)).__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
index a785c0c..acf040e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.webapp.MimeType;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.WebAppException;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public abstract class HtmlBlock extends TextView implements SubView {


---------------------------------------------------------------------
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: HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10630. Federation State Store FS Implementation. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 13f300e0db92368cedace0d3534415898a9cff52
Parents: 1e50322
Author: Inigo Goiri <in...@apache.org>
Authored: Tue May 2 15:49:53 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 28 09:48:38 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  14 +
 .../federation/router/PeriodicService.java      | 198 ++++++++
 .../StateStoreConnectionMonitorService.java     |  67 +++
 .../federation/store/StateStoreService.java     | 152 +++++-
 .../federation/store/StateStoreUtils.java       |  51 +-
 .../store/driver/StateStoreDriver.java          |  31 +-
 .../driver/StateStoreRecordOperations.java      |  17 +-
 .../store/driver/impl/StateStoreBaseImpl.java   |  31 +-
 .../driver/impl/StateStoreFileBaseImpl.java     | 429 ++++++++++++++++
 .../store/driver/impl/StateStoreFileImpl.java   | 161 +++++++
 .../driver/impl/StateStoreFileSystemImpl.java   | 178 +++++++
 .../driver/impl/StateStoreSerializableImpl.java |  77 +++
 .../federation/store/records/BaseRecord.java    |  20 +-
 .../server/federation/store/records/Query.java  |  66 +++
 .../src/main/resources/hdfs-default.xml         |  16 +
 .../store/FederationStateStoreTestUtils.java    | 232 +++++++++
 .../store/driver/TestStateStoreDriverBase.java  | 483 +++++++++++++++++++
 .../store/driver/TestStateStoreFile.java        |  64 +++
 .../store/driver/TestStateStoreFileSystem.java  |  88 ++++
 19 files changed, 2329 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index cd7f1a8..97a6e59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -25,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl;
 import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
 import org.apache.hadoop.http.HttpConfig;
 
@@ -1115,6 +1119,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT =
           StateStoreSerializerPBImpl.class;
 
+  public static final String FEDERATION_STORE_DRIVER_CLASS =
+      FEDERATION_STORE_PREFIX + "driver.class";
+  public static final Class<? extends StateStoreDriver>
+      FEDERATION_STORE_DRIVER_CLASS_DEFAULT = StateStoreFileImpl.class;
+
+  public static final String FEDERATION_STORE_CONNECTION_TEST_MS =
+      FEDERATION_STORE_PREFIX + "connection.test";
+  public static final long FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT =
+      TimeUnit.MINUTES.toMillis(1);
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.java
new file mode 100644
index 0000000..5e12222
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/PeriodicService.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.hdfs.server.federation.router;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Service to periodically execute a runnable.
+ */
+public abstract class PeriodicService extends AbstractService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PeriodicService.class);
+
+  /** Default interval in milliseconds for the periodic service. */
+  private static final long DEFAULT_INTERVAL_MS = TimeUnit.MINUTES.toMillis(1);
+
+
+  /** Interval for running the periodic service in milliseconds. */
+  private long intervalMs;
+  /** Name of the service. */
+  private final String serviceName;
+
+  /** Scheduler for the periodic service. */
+  private final ScheduledExecutorService scheduler;
+
+  /** If the service is running. */
+  private volatile boolean isRunning = false;
+
+  /** How many times we run. */
+  private long runCount;
+  /** How many errors we got. */
+  private long errorCount;
+  /** When was the last time we executed this service successfully. */
+  private long lastRun;
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   */
+  public PeriodicService(String name) {
+    this(name, DEFAULT_INTERVAL_MS);
+  }
+
+  /**
+   * Create a new periodic update service.
+   *
+   * @param name Name of the service.
+   * @param interval Interval for the periodic service in milliseconds.
+   */
+  public PeriodicService(String name, long interval) {
+    super(name);
+    this.serviceName = name;
+    this.intervalMs = interval;
+
+    ThreadFactory threadFactory = new ThreadFactoryBuilder()
+        .setNameFormat(this.getName() + "-%d")
+        .build();
+    this.scheduler = Executors.newScheduledThreadPool(1, threadFactory);
+  }
+
+  /**
+   * Set the interval for the periodic service.
+   *
+   * @param interval Interval in milliseconds.
+   */
+  protected void setIntervalMs(long interval) {
+    if (getServiceState() == STATE.STARTED) {
+      throw new ServiceStateException("Periodic service already started");
+    } else {
+      this.intervalMs = interval;
+    }
+  }
+
+  /**
+   * Get the interval for the periodic service.
+   *
+   * @return Interval in milliseconds.
+   */
+  protected long getIntervalMs() {
+    return this.intervalMs;
+  }
+
+  /**
+   * Get how many times we failed to run the periodic service.
+   *
+   * @return Times we failed to run the periodic service.
+   */
+  protected long getErrorCount() {
+    return this.errorCount;
+  }
+
+  /**
+   * Get how many times we run the periodic service.
+   *
+   * @return Times we run the periodic service.
+   */
+  protected long getRunCount() {
+    return this.runCount;
+  }
+
+  /**
+   * Get the last time the periodic service was executed.
+   *
+   * @return Last time the periodic service was executed.
+   */
+  protected long getLastUpdate() {
+    return this.lastRun;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    LOG.info("Starting periodic service {}", this.serviceName);
+    startPeriodic();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    stopPeriodic();
+    LOG.info("Stopping periodic service {}", this.serviceName);
+    super.serviceStop();
+  }
+
+  /**
+   * Stop the periodic task.
+   */
+  protected synchronized void stopPeriodic() {
+    if (this.isRunning) {
+      LOG.info("{} is shutting down", this.serviceName);
+      this.isRunning = false;
+      this.scheduler.shutdownNow();
+    }
+  }
+
+  /**
+   * Start the periodic execution.
+   */
+  protected synchronized void startPeriodic() {
+    stopPeriodic();
+
+    // Create the runnable service
+    Runnable updateRunnable = new Runnable() {
+      @Override
+      public void run() {
+        LOG.debug("Running {} update task", serviceName);
+        try {
+          if (!isRunning) {
+            return;
+          }
+          periodicInvoke();
+          runCount++;
+          lastRun = Time.now();
+        } catch (Exception ex) {
+          errorCount++;
+          LOG.warn(serviceName + " service threw an exception", ex);
+        }
+      }
+    };
+
+    // Start the execution of the periodic service
+    this.isRunning = true;
+    this.scheduler.scheduleWithFixedDelay(
+        updateRunnable, 0, this.intervalMs, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Method that the service will run periodically.
+   */
+  protected abstract void periodicInvoke();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.java
new file mode 100644
index 0000000..4d279c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreConnectionMonitorService.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.hdfs.server.federation.store;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.router.PeriodicService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service to periodically monitor the connection of the StateStore
+ * {@link StateStoreService} data store and to re-open the connection
+ * to the data store if required.
+ */
+public class StateStoreConnectionMonitorService extends PeriodicService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreConnectionMonitorService.class);
+
+  /** Service that maintains the State Store connection. */
+  private final StateStoreService stateStore;
+
+
+  /**
+   * Create a new service to monitor the connectivity of the state store driver.
+   *
+   * @param store Instance of the state store to be monitored.
+   */
+  public StateStoreConnectionMonitorService(StateStoreService store) {
+    super(StateStoreConnectionMonitorService.class.getSimpleName());
+    this.stateStore = store;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    this.setIntervalMs(conf.getLong(
+        DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS,
+        DFSConfigKeys.FEDERATION_STORE_CONNECTION_TEST_MS_DEFAULT));
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void periodicInvoke() {
+    LOG.debug("Checking state store connection");
+    if (!stateStore.isDriverReady()) {
+      LOG.info("Attempting to open state store driver.");
+      stateStore.loadDriver();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
index 866daa3..df207e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
@@ -15,45 +15,168 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hdfs.server.federation.store;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * A service to initialize a
  * {@link org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver
- * StateStoreDriver} and maintain the connection to the data store. There
- * are multiple state store driver connections supported:
+ * StateStoreDriver} and maintain the connection to the data store. There are
+ * multiple state store driver connections supported:
  * <ul>
- * <li>File {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * <li>File
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
  * StateStoreFileImpl StateStoreFileImpl}
- * <li>ZooKeeper {@link org.apache.hadoop.hdfs.server.federation.store.driver.
- * impl.StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
+ * <li>ZooKeeper
+ * {@link org.apache.hadoop.hdfs.server.federation.store.driver.impl.
+ * StateStoreZooKeeperImpl StateStoreZooKeeperImpl}
  * </ul>
  * <p>
- * The service also supports the dynamic registration of data interfaces such as
- * the following:
+ * The service also supports the dynamic registration of record stores like:
  * <ul>
- * <li>{@link MembershipStateStore}: state of the Namenodes in the
+ * <li>{@link MembershipStore}: state of the Namenodes in the
  * federation.
  * <li>{@link MountTableStore}: Mount table between to subclusters.
  * See {@link org.apache.hadoop.fs.viewfs.ViewFs ViewFs}.
- * <li>{@link RouterStateStore}: State of the routers in the federation.
  * </ul>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class StateStoreService extends CompositeService {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreService.class);
+
+
+  /** State Store configuration. */
+  private Configuration conf;
+
   /** Identifier for the service. */
   private String identifier;
 
-  // Stub class
-  public StateStoreService(String name) {
-    super(name);
+  /** Driver for the back end connection. */
+  private StateStoreDriver driver;
+
+  /** Service to maintain data store connection. */
+  private StateStoreConnectionMonitorService monitorService;
+
+
+  public StateStoreService() {
+    super(StateStoreService.class.getName());
+  }
+
+  /**
+   * Initialize the State Store and the connection to the backend.
+   *
+   * @param config Configuration for the State Store.
+   * @throws IOException
+   */
+  @Override
+  protected void serviceInit(Configuration config) throws Exception {
+    this.conf = config;
+
+    // Create implementation of State Store
+    Class<? extends StateStoreDriver> driverClass = this.conf.getClass(
+        DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS,
+        DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS_DEFAULT,
+        StateStoreDriver.class);
+    this.driver = ReflectionUtils.newInstance(driverClass, this.conf);
+
+    if (this.driver == null) {
+      throw new IOException("Cannot create driver for the State Store");
+    }
+
+    // Check the connection to the State Store periodically
+    this.monitorService = new StateStoreConnectionMonitorService(this);
+    this.addService(monitorService);
+
+    super.serviceInit(this.conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    loadDriver();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    closeDriver();
+
+    super.serviceStop();
+  }
+
+  /**
+   * List of records supported by this State Store.
+   *
+   * @return List of supported record classes.
+   */
+  public Collection<Class<? extends BaseRecord>> getSupportedRecords() {
+    // TODO add list of records
+    return new LinkedList<>();
+  }
+
+  /**
+   * Load the State Store driver. If successful, refresh cached data tables.
+   */
+  public void loadDriver() {
+    synchronized (this.driver) {
+      if (!isDriverReady()) {
+        String driverName = this.driver.getClass().getSimpleName();
+        if (this.driver.init(conf, getIdentifier(), getSupportedRecords())) {
+          LOG.info("Connection to the State Store driver {} is open and ready",
+              driverName);
+        } else {
+          LOG.error("Cannot initialize State Store driver {}", driverName);
+        }
+      }
+    }
+  }
+
+  /**
+   * Check if the driver is ready to be used.
+   *
+   * @return If the driver is ready.
+   */
+  public boolean isDriverReady() {
+    return this.driver.isDriverReady();
+  }
+
+  /**
+   * Manually shuts down the driver.
+   *
+   * @throws Exception If the driver cannot be closed.
+   */
+  @VisibleForTesting
+  public void closeDriver() throws Exception {
+    if (this.driver != null) {
+      this.driver.close();
+    }
+  }
+
+  /**
+   * Get the state store driver.
+   *
+   * @return State store driver.
+   */
+  public StateStoreDriver getDriver() {
+    return this.driver;
   }
 
   /**
@@ -74,4 +197,5 @@ public class StateStoreService extends CompositeService {
   public void setIdentifier(String id) {
     this.identifier = id;
   }
-}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
index 8c681df..0a36619 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreUtils.java
@@ -17,17 +17,22 @@
  */
 package org.apache.hadoop.hdfs.server.federation.store;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Set of utility functions used to query, create, update and delete data
- * records in the state store.
+ * Set of utility functions used to work with the State Store.
  */
 public final class StateStoreUtils {
 
-  private static final Log LOG = LogFactory.getLog(StateStoreUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreUtils.class);
+
 
   private StateStoreUtils() {
     // Utility class
@@ -52,7 +57,7 @@ public final class StateStoreUtils {
 
     // Check if we went too far
     if (actualClazz.equals(BaseRecord.class)) {
-      LOG.error("We went too far (" + actualClazz + ") with " + clazz);
+      LOG.error("We went too far ({}) with {}", actualClazz, clazz);
       actualClazz = clazz;
     }
     return actualClazz;
@@ -69,4 +74,36 @@ public final class StateStoreUtils {
       Class<? extends BaseRecord> getRecordClass(final T record) {
     return getRecordClass(record.getClass());
   }
-}
+
+  /**
+   * Get the base class name for a record. If we get an implementation of a
+   * record we will return the real parent record class.
+   *
+   * @param clazz Class of the data record to check.
+   * @return Name of the base class for the record.
+   */
+  public static <T extends BaseRecord> String getRecordName(
+      final Class<T> clazz) {
+    return getRecordClass(clazz).getSimpleName();
+  }
+
+  /**
+   * Filters a list of records to find all records matching the query.
+   *
+   * @param query Map of field names and objects to use to filter results.
+   * @param records List of data records to filter.
+   * @return List of all records matching the query (or empty list if none
+   *         match), null if the data set could not be filtered.
+   */
+  public static <T extends BaseRecord> List<T> filterMultiple(
+      final Query<T> query, final Iterable<T> records) {
+
+    List<T> matchingList = new ArrayList<>();
+    for (T record : records) {
+      if (query.matches(record)) {
+        matchingList.add(record);
+      }
+    }
+    return matchingList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
index a1527df..90111bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
@@ -18,15 +18,16 @@
 package org.apache.hadoop.hdfs.server.federation.store.driver;
 
 import java.net.InetAddress;
-import java.util.List;
+import java.util.Collection;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
 import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Driver class for an implementation of a {@link StateStoreService}
@@ -35,7 +36,8 @@ import org.apache.hadoop.util.Time;
  */
 public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
-  private static final Log LOG = LogFactory.getLog(StateStoreDriver.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreDriver.class);
 
 
   /** State Store configuration. */
@@ -47,13 +49,14 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
   /**
    * Initialize the state store connection.
+   *
    * @param config Configuration for the driver.
    * @param id Identifier for the driver.
    * @param records Records that are supported.
    * @return If initialized and ready, false if failed to initialize driver.
    */
   public boolean init(final Configuration config, final String id,
-      final List<Class<? extends BaseRecord>> records) {
+      final Collection<Class<? extends BaseRecord>> records) {
 
     this.conf = config;
     this.identifier = id;
@@ -62,8 +65,20 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
       LOG.warn("The identifier for the State Store connection is not set");
     }
 
-    // TODO stub
-    return false;
+    boolean success = initDriver();
+    if (!success) {
+      LOG.error("Cannot intialize driver for {}", getDriverName());
+      return false;
+    }
+
+    for (Class<? extends BaseRecord> cls : records) {
+      String recordString = StateStoreUtils.getRecordName(cls);
+      if (!initRecordStorage(recordString, cls)) {
+        LOG.error("Cannot initialize record store for {}", cls.getSimpleName());
+        return false;
+      }
+    }
+    return true;
   }
 
   /**
@@ -169,4 +184,4 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
     }
     return hostname;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
index 739eeba..e76a733 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.federation.store.driver;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
 import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
@@ -67,14 +67,14 @@ public interface StateStoreRecordOperations {
    * Get a single record from the store that matches the query.
    *
    * @param clazz Class of record to fetch.
-   * @param query Map of field names and objects to filter results.
+   * @param query Query to filter results.
    * @return A single record matching the query. Null if there are no matching
    *         records or more than one matching record in the store.
    * @throws IOException If multiple records match or if the data store cannot
    *           be queried.
    */
   @Idempotent
-  <T extends BaseRecord> T get(Class<T> clazz, Map<String, String> query)
+  <T extends BaseRecord> T get(Class<T> clazz, Query<T> query)
       throws IOException;
 
   /**
@@ -83,14 +83,14 @@ public interface StateStoreRecordOperations {
    * supports filtering it should overwrite this method.
    *
    * @param clazz Class of record to fetch.
-   * @param query Map of field names and objects to filter results.
+   * @param query Query to filter results.
    * @return Records of type clazz that match the query or empty list if none
    *         are found.
    * @throws IOException Throws exception if unable to query the data store.
    */
   @Idempotent
   <T extends BaseRecord> List<T> getMultiple(
-      Class<T> clazz, Map<String, String> query) throws IOException;
+      Class<T> clazz, Query<T> query) throws IOException;
 
   /**
    * Creates a single record. Optionally updates an existing record with same
@@ -152,13 +152,12 @@ public interface StateStoreRecordOperations {
    * Remove multiple records of a specific class that match a query. Requires
    * the getAll implementation to fetch fresh records on each call.
    *
-   * @param clazz Class of record to remove.
-   * @param filter matching filter to remove.
+   * @param query Query to filter what to remove.
    * @return The number of records removed.
    * @throws IOException Throws exception if unable to query the data store.
    */
   @AtMostOnce
-  <T extends BaseRecord> int remove(Class<T> clazz, Map<String, String> filter)
+  <T extends BaseRecord> int remove(Class<T> clazz, Query<T> query)
       throws IOException;
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
index b711fa9..1bd35f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreBaseImpl.java
@@ -17,14 +17,17 @@
  */
 package org.apache.hadoop.hdfs.server.federation.store.driver.impl;
 
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
 
 /**
  * Base implementation of a State Store driver. It contains default
@@ -41,7 +44,7 @@ public abstract class StateStoreBaseImpl extends StateStoreDriver {
 
   @Override
   public <T extends BaseRecord> T get(
-      Class<T> clazz, Map<String, String> query) throws IOException {
+      Class<T> clazz, Query<T> query) throws IOException {
     List<T> records = getMultiple(clazz, query);
     if (records.size() > 1) {
       throw new IOException("Found more than one object in collection");
@@ -53,17 +56,31 @@ public abstract class StateStoreBaseImpl extends StateStoreDriver {
   }
 
   @Override
+  public <T extends BaseRecord> List<T> getMultiple(
+      Class<T> clazz, Query<T> query) throws IOException  {
+    QueryResult<T> result = get(clazz);
+    List<T> records = result.getRecords();
+    List<T> ret = filterMultiple(query, records);
+    if (ret == null) {
+      throw new IOException("Cannot fetch records from the store");
+    }
+    return ret;
+  }
+
+  @Override
   public <T extends BaseRecord> boolean put(
       T record, boolean allowUpdate, boolean errorIfExists) throws IOException {
-    List<T> singletonList = new ArrayList<T>();
+    List<T> singletonList = new ArrayList<>();
     singletonList.add(record);
     return putAll(singletonList, allowUpdate, errorIfExists);
   }
 
   @Override
   public <T extends BaseRecord> boolean remove(T record) throws IOException {
-    Map<String, String> primaryKeys = record.getPrimaryKeys();
-    Class<? extends BaseRecord> clazz = StateStoreUtils.getRecordClass(record);
-    return remove(clazz, primaryKeys) == 1;
+    final Query<T> query = new Query<T>(record);
+    Class<? extends BaseRecord> clazz = record.getClass();
+    @SuppressWarnings("unchecked")
+    Class<T> recordClass = (Class<T>)StateStoreUtils.getRecordClass(clazz);
+    return remove(recordClass, query) == 1;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
new file mode 100644
index 0000000..d7c00ff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
@@ -0,0 +1,429 @@
+/**
+ * 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.store.driver.impl;
+
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.filterMultiple;
+import static org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils.getRecordClass;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUnavailableException;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.Query;
+import org.apache.hadoop.hdfs.server.federation.store.records.QueryResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link StateStoreDriver} implementation based on a local file.
+ */
+public abstract class StateStoreFileBaseImpl
+    extends StateStoreSerializableImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileBaseImpl.class);
+
+  /** If it is initialized. */
+  private boolean initialized = false;
+
+  /** Name of the file containing the data. */
+  private static final String DATA_FILE_NAME = "records.data";
+
+
+  /**
+   * Lock reading records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void lockRecordRead(Class<T> clazz);
+
+  /**
+   * Unlock reading records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void unlockRecordRead(
+      Class<T> clazz);
+
+  /**
+   * Lock writing records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void lockRecordWrite(
+      Class<T> clazz);
+
+  /**
+   * Unlock writing records.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> void unlockRecordWrite(
+      Class<T> clazz);
+
+  /**
+   * Get the reader for the file system.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub);
+
+  /**
+   * Get the writer for the file system.
+   *
+   * @param clazz Class of the record.
+   */
+  protected abstract <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub);
+
+  /**
+   * Check if a path exists.
+   *
+   * @param path Path to check.
+   * @return If the path exists.
+   */
+  protected abstract boolean exists(String path);
+
+  /**
+   * Make a directory.
+   *
+   * @param path Path of the directory to create.
+   * @return If the directory was created.
+   */
+  protected abstract boolean mkdir(String path);
+
+  /**
+   * Get root directory.
+   *
+   * @return Root directory.
+   */
+  protected abstract String getRootDir();
+
+  /**
+   * Set the driver as initialized.
+   *
+   * @param ini If the driver is initialized.
+   */
+  public void setInitialized(boolean ini) {
+    this.initialized = ini;
+  }
+
+  @Override
+  public boolean initDriver() {
+    String rootDir = getRootDir();
+    try {
+      if (rootDir == null) {
+        LOG.error("Invalid root directory, unable to initialize driver.");
+        return false;
+      }
+
+      // Check root path
+      if (!exists(rootDir)) {
+        if (!mkdir(rootDir)) {
+          LOG.error("Cannot create State Store root directory {}", rootDir);
+          return false;
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error(
+          "Cannot initialize filesystem using root directory {}", rootDir, ex);
+      return false;
+    }
+    setInitialized(true);
+    return true;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean initRecordStorage(
+      String className, Class<T> recordClass) {
+
+    String dataDirPath = getRootDir() + "/" + className;
+    try {
+      // Create data directories for files
+      if (!exists(dataDirPath)) {
+        LOG.info("{} data directory doesn't exist, creating it", dataDirPath);
+        if (!mkdir(dataDirPath)) {
+          LOG.error("Cannot create data directory {}", dataDirPath);
+          return false;
+        }
+        String dataFilePath = dataDirPath + "/" + DATA_FILE_NAME;
+        if (!exists(dataFilePath)) {
+          // Create empty file
+          List<T> emtpyList = new ArrayList<>();
+          if(!writeAll(emtpyList, recordClass)) {
+            LOG.error("Cannot create data file {}", dataFilePath);
+            return false;
+          }
+        }
+      }
+    } catch (Exception ex) {
+      LOG.error("Cannot create data directory {}", dataDirPath, ex);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Read all lines from a file and deserialize into the desired record type.
+   *
+   * @param reader Open handle for the file.
+   * @param recordClass Record class to create.
+   * @param includeDates True if dateModified/dateCreated are serialized.
+   * @return List of records.
+   * @throws IOException
+   */
+  private <T extends BaseRecord> List<T> getAllFile(
+      BufferedReader reader, Class<T> clazz, boolean includeDates)
+          throws IOException {
+
+    List<T> ret = new ArrayList<T>();
+    String line;
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("#") && line.length() > 0) {
+        try {
+          T record = newRecord(line, clazz, includeDates);
+          ret.add(record);
+        } catch (Exception ex) {
+          LOG.error("Cannot parse line in data source file: {}", line, ex);
+        }
+      }
+    }
+    return ret;
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz)
+      throws IOException {
+    return get(clazz, (String)null);
+  }
+
+  @Override
+  public <T extends BaseRecord> QueryResult<T> get(Class<T> clazz, String sub)
+      throws IOException {
+    verifyDriverReady();
+    BufferedReader reader = null;
+    lockRecordRead(clazz);
+    try {
+      reader = getReader(clazz, sub);
+      List<T> data = getAllFile(reader, clazz, true);
+      return new QueryResult<T>(data, getTime());
+    } catch (Exception ex) {
+      LOG.error("Cannot fetch records {}", clazz.getSimpleName());
+      throw new IOException("Cannot read from data store " + ex.getMessage());
+    } finally {
+      if (reader != null) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Failed closing file", e);
+        }
+      }
+      unlockRecordRead(clazz);
+    }
+  }
+
+  /**
+   * Overwrite the existing data with a new data set.
+   *
+   * @param list List of records to write.
+   * @param writer BufferedWriter stream to write to.
+   * @return If the records were succesfully written.
+   */
+  private <T extends BaseRecord> boolean writeAllFile(
+      Collection<T> records, BufferedWriter writer) {
+
+    try {
+      for (BaseRecord record : records) {
+        try {
+          String data = serializeString(record);
+          writer.write(data);
+          writer.newLine();
+        } catch (IllegalArgumentException ex) {
+          LOG.error("Cannot write record {} to file", record, ex);
+        }
+      }
+      writer.flush();
+      return true;
+    } catch (IOException e) {
+      LOG.error("Cannot commit records to file", e);
+      return false;
+    }
+  }
+
+  /**
+   * Overwrite the existing data with a new data set. Replaces all records in
+   * the data store for this record class. If all records in the data store are
+   * not successfully committed, this function must return false and leave the
+   * data store unchanged.
+   *
+   * @param records List of records to write. All records must be of type
+   *                recordClass.
+   * @param recordClass Class of record to replace.
+   * @return true if all operations were successful, false otherwise.
+   * @throws StateStoreUnavailableException
+   */
+  public <T extends BaseRecord> boolean writeAll(
+      Collection<T> records, Class<T> recordClass)
+          throws StateStoreUnavailableException {
+    verifyDriverReady();
+    lockRecordWrite(recordClass);
+    BufferedWriter writer = null;
+    try {
+      writer = getWriter(recordClass, null);
+      return writeAllFile(records, writer);
+    } catch (Exception e) {
+      LOG.error(
+          "Cannot add records to file for {}", recordClass.getSimpleName(), e);
+      return false;
+    } finally {
+      if (writer != null) {
+        try {
+          writer.close();
+        } catch (IOException e) {
+          LOG.error(
+              "Cannot close writer for {}", recordClass.getSimpleName(), e);
+        }
+      }
+      unlockRecordWrite(recordClass);
+    }
+  }
+
+  /**
+   * Get the data file name.
+   *
+   * @return Data file name.
+   */
+  protected String getDataFileName() {
+    return DATA_FILE_NAME;
+  }
+
+  @Override
+  public boolean isDriverReady() {
+    return this.initialized;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean putAll(
+      List<T> records, boolean allowUpdate, boolean errorIfExists)
+          throws StateStoreUnavailableException {
+    verifyDriverReady();
+
+    if (records.isEmpty()) {
+      return true;
+    }
+
+    @SuppressWarnings("unchecked")
+    Class<T> clazz = (Class<T>) getRecordClass(records.get(0).getClass());
+    QueryResult<T> result;
+    try {
+      result = get(clazz);
+    } catch (IOException e) {
+      return false;
+    }
+    Map<Object, T> writeList = new HashMap<>();
+
+    // Write all of the existing records
+    for (T existingRecord : result.getRecords()) {
+      String key = existingRecord.getPrimaryKey();
+      writeList.put(key, existingRecord);
+    }
+
+    // Add inserts and updates, overwrite any existing values
+    for (T updatedRecord : records) {
+      try {
+        updatedRecord.validate();
+        String key = updatedRecord.getPrimaryKey();
+        if (writeList.containsKey(key) && allowUpdate) {
+          // Update
+          writeList.put(key, updatedRecord);
+          // Update the mod time stamp. Many backends will use their
+          // own timestamp for the mod time.
+          updatedRecord.setDateModified(this.getTime());
+        } else if (!writeList.containsKey(key)) {
+          // Insert
+          // Create/Mod timestamps are already initialized
+          writeList.put(key, updatedRecord);
+        } else if (errorIfExists) {
+          LOG.error("Attempt to insert record {} that already exists",
+              updatedRecord);
+          return false;
+        }
+      } catch (IllegalArgumentException ex) {
+        LOG.error("Cannot write invalid record to State Store", ex);
+        return false;
+      }
+    }
+
+    // Write all
+    boolean status = writeAll(writeList.values(), clazz);
+    return status;
+  }
+
+  @Override
+  public <T extends BaseRecord> int remove(Class<T> clazz, Query<T> query)
+      throws StateStoreUnavailableException {
+    verifyDriverReady();
+
+    if (query == null) {
+      return 0;
+    }
+
+    int removed = 0;
+    // Get the current records
+    try {
+      final QueryResult<T> result = get(clazz);
+      final List<T> existingRecords = result.getRecords();
+      // Write all of the existing records except those to be removed
+      final List<T> recordsToRemove = filterMultiple(query, existingRecords);
+      removed = recordsToRemove.size();
+      final List<T> newRecords = new LinkedList<>();
+      for (T record : existingRecords) {
+        if (!recordsToRemove.contains(record)) {
+          newRecords.add(record);
+        }
+      }
+      if (!writeAll(newRecords, clazz)) {
+        throw new IOException(
+            "Cannot remove record " + clazz + " query " + query);
+      }
+    } catch (IOException e) {
+      LOG.error("Cannot remove records {} query {}", clazz, query, e);
+    }
+
+    return removed;
+  }
+
+  @Override
+  public <T extends BaseRecord> boolean removeAll(Class<T> clazz)
+      throws StateStoreUnavailableException {
+    verifyDriverReady();
+    List<T> emptyList = new ArrayList<>();
+    boolean status = writeAll(emptyList, clazz);
+    return status;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
new file mode 100644
index 0000000..24e9660
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileImpl.java
@@ -0,0 +1,161 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.Files;
+
+/**
+ * StateStoreDriver implementation based on a local file.
+ */
+public class StateStoreFileImpl extends StateStoreFileBaseImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileImpl.class);
+
+  /** Configuration keys. */
+  public static final String FEDERATION_STORE_FILE_DIRECTORY =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "driver.file.directory";
+
+  /** Synchronization. */
+  private static final ReadWriteLock READ_WRITE_LOCK =
+      new ReentrantReadWriteLock();
+
+  /** Root directory for the state store. */
+  private String rootDirectory;
+
+
+  @Override
+  protected boolean exists(String path) {
+    File test = new File(path);
+    return test.exists();
+  }
+
+  @Override
+  protected boolean mkdir(String path) {
+    File dir = new File(path);
+    return dir.mkdirs();
+  }
+
+  @Override
+  protected String getRootDir() {
+    if (this.rootDirectory == null) {
+      String dir = getConf().get(FEDERATION_STORE_FILE_DIRECTORY);
+      if (dir == null) {
+        File tempDir = Files.createTempDir();
+        dir = tempDir.getAbsolutePath();
+      }
+      this.rootDirectory = dir;
+    }
+    return this.rootDirectory;
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordWrite(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.writeLock().lock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordWrite(
+      Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.writeLock().unlock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordRead(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.readLock().lock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordRead(Class<T> recordClass) {
+    // TODO - Synchronize via FS
+    READ_WRITE_LOCK.readLock().unlock();
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub) {
+    String filename = StateStoreUtils.getRecordName(clazz);
+    if (sub != null && sub.length() > 0) {
+      filename += "/" + sub;
+    }
+    filename += "/" + getDataFileName();
+
+    try {
+      LOG.debug("Loading file: {}", filename);
+      File file = new File(getRootDir(), filename);
+      FileInputStream fis = new FileInputStream(file);
+      InputStreamReader isr =
+          new InputStreamReader(fis, StandardCharsets.UTF_8);
+      BufferedReader reader = new BufferedReader(isr);
+      return reader;
+    } catch (Exception ex) {
+      LOG.error(
+          "Cannot open read stream for record {}", clazz.getSimpleName(), ex);
+      return null;
+    }
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub) {
+    String filename = StateStoreUtils.getRecordName(clazz);
+    if (sub != null && sub.length() > 0) {
+      filename += "/" + sub;
+    }
+    filename += "/" + getDataFileName();
+
+    try {
+      File file = new File(getRootDir(), filename);
+      FileOutputStream fos = new FileOutputStream(file, false);
+      OutputStreamWriter osw =
+          new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+      BufferedWriter writer = new BufferedWriter(osw);
+      return writer;
+    } catch (IOException ex) {
+      LOG.error(
+          "Cannot open read stream for record {}", clazz.getSimpleName(), ex);
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    setInitialized(false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
new file mode 100644
index 0000000..5968421
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileSystemImpl.java
@@ -0,0 +1,178 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreUtils;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StateStoreDriver} implementation based on a filesystem. The most common uses
+ * HDFS as a backend.
+ */
+public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StateStoreFileSystemImpl.class);
+
+
+  /** Configuration keys. */
+  public static final String FEDERATION_STORE_FS_PATH =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "driver.fs.path";
+
+  /** File system to back the State Store. */
+  private FileSystem fs;
+  /** Working path in the filesystem. */
+  private String workPath;
+
+  @Override
+  protected boolean exists(String path) {
+    try {
+      return fs.exists(new Path(path));
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  @Override
+  protected boolean mkdir(String path) {
+    try {
+      return fs.mkdirs(new Path(path));
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  @Override
+  protected String getRootDir() {
+    if (this.workPath == null) {
+      String rootPath = getConf().get(FEDERATION_STORE_FS_PATH);
+      URI workUri;
+      try {
+        workUri = new URI(rootPath);
+        fs = FileSystem.get(workUri, getConf());
+      } catch (Exception ex) {
+        return null;
+      }
+      this.workPath = rootPath;
+    }
+    return this.workPath;
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+  }
+
+  /**
+   * Get the folder path for the record class' data.
+   *
+   * @param cls Data record class.
+   * @return Path of the folder containing the record class' data files.
+   */
+  private Path getPathForClass(Class<? extends BaseRecord> clazz) {
+    if (clazz == null) {
+      return null;
+    }
+    // TODO extract table name from class: entry.getTableName()
+    String className = StateStoreUtils.getRecordName(clazz);
+    return new Path(workPath, className);
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordRead(Class<T> clazz) {
+    // Not required, synced with HDFS leasing
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordRead(Class<T> clazz) {
+    // Not required, synced with HDFS leasing
+  }
+
+  @Override
+  protected <T extends BaseRecord> void lockRecordWrite(Class<T> clazz) {
+    // TODO -> wait for lease to be available
+  }
+
+  @Override
+  protected <T extends BaseRecord> void unlockRecordWrite(Class<T> clazz) {
+    // TODO -> ensure lease is closed for the file
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedReader getReader(
+      Class<T> clazz, String sub) {
+
+    Path path = getPathForClass(clazz);
+    if (sub != null && sub.length() > 0) {
+      path = Path.mergePaths(path, new Path("/" + sub));
+    }
+    path = Path.mergePaths(path, new Path("/" + getDataFileName()));
+
+    try {
+      FSDataInputStream fdis = fs.open(path);
+      InputStreamReader isr =
+          new InputStreamReader(fdis, StandardCharsets.UTF_8);
+      BufferedReader reader = new BufferedReader(isr);
+      return reader;
+    } catch (IOException ex) {
+      LOG.error("Cannot open write stream for {}  to {}",
+          clazz.getSimpleName(), path);
+      return null;
+    }
+  }
+
+  @Override
+  protected <T extends BaseRecord> BufferedWriter getWriter(
+      Class<T> clazz, String sub) {
+
+    Path path = getPathForClass(clazz);
+    if (sub != null && sub.length() > 0) {
+      path = Path.mergePaths(path, new Path("/" + sub));
+    }
+    path = Path.mergePaths(path, new Path("/" + getDataFileName()));
+
+    try {
+      FSDataOutputStream fdos = fs.create(path, true);
+      OutputStreamWriter osw =
+          new OutputStreamWriter(fdos, StandardCharsets.UTF_8);
+      BufferedWriter writer = new BufferedWriter(osw);
+      return writer;
+    } catch (IOException ex) {
+      LOG.error("Cannot open write stream for {} to {}",
+          clazz.getSimpleName(), path);
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
new file mode 100644
index 0000000..e9b3fdf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.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.hdfs.server.federation.store.driver.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * State Store driver that stores a serialization of the records. The serializer
+ * is pluggable.
+ */
+public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
+
+  /** Default serializer for this driver. */
+  private StateStoreSerializer serializer;
+
+
+  @Override
+  public boolean init(final Configuration config, final String id,
+      final Collection<Class<? extends BaseRecord>> records) {
+    boolean ret = super.init(config, id, records);
+
+    this.serializer = StateStoreSerializer.getSerializer(config);
+
+    return ret;
+  }
+
+  /**
+   * Serialize a record using the serializer.
+   * @param record Record to serialize.
+   * @return Byte array with the serialization of the record.
+   */
+  protected <T extends BaseRecord> byte[] serialize(T record) {
+    return serializer.serialize(record);
+  }
+
+  /**
+   * Serialize a record using the serializer.
+   * @param record Record to serialize.
+   * @return String with the serialization of the record.
+   */
+  protected <T extends BaseRecord> String serializeString(T record) {
+    return serializer.serializeString(record);
+  }
+
+  /**
+   * Creates a record from an input data string.
+   * @param data Serialized text of the record.
+   * @param clazz Record class.
+   * @param includeDates If dateModified and dateCreated are serialized.
+   * @return The created record.
+   * @throws IOException
+   */
+  protected <T extends BaseRecord> T newRecord(
+      String data, Class<T> clazz, boolean includeDates) throws IOException {
+    return serializer.deserialize(data, clazz);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
index 4192a3d..79f99c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/BaseRecord.java
@@ -123,6 +123,24 @@ public abstract class BaseRecord implements Comparable<BaseRecord> {
   }
 
   /**
+   * Check if this record matches a partial record.
+   *
+   * @param other Partial record.
+   * @return If this record matches.
+   */
+  public boolean like(BaseRecord other) {
+    if (other == null) {
+      return false;
+    }
+    Map<String, String> thisKeys = this.getPrimaryKeys();
+    Map<String, String> otherKeys = other.getPrimaryKeys();
+    if (thisKeys == null) {
+      return otherKeys == null;
+    }
+    return thisKeys.equals(otherKeys);
+  }
+
+  /**
    * Override equals check to use primary key(s) for comparison.
    */
   @Override
@@ -186,4 +204,4 @@ public abstract class BaseRecord implements Comparable<BaseRecord> {
   public String toString() {
     return getPrimaryKey();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
new file mode 100644
index 0000000..3c59abf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/Query.java
@@ -0,0 +1,66 @@
+/**
+ * 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.store.records;
+
+/**
+ * Check if a record matches a query. The query is usually a partial record.
+ *
+ * @param <T> Type of the record to query.
+ */
+public class Query<T extends BaseRecord> {
+
+  /** Partial object to compare against. */
+  private final T partial;
+
+
+  /**
+   * Create a query to search for a partial record.
+   *
+   * @param partial It defines the attributes to search.
+   */
+  public Query(final T part) {
+    this.partial = part;
+  }
+
+  /**
+   * Get the partial record used to query.
+   *
+   * @return The partial record used for the query.
+   */
+  public T getPartial() {
+    return this.partial;
+  }
+
+  /**
+   * Check if a record matches the primary keys or the partial record.
+   *
+   * @param other Record to check.
+   * @return If the record matches. Don't match if there is no partial.
+   */
+  public boolean matches(T other) {
+    if (this.partial == null) {
+      return false;
+    }
+    return this.partial.like(other);
+  }
+
+  @Override
+  public String toString() {
+    return "Checking: " + this.partial;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 39fbf98..cfb029b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4567,4 +4567,20 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.store.driver.class</name>
+    <value>org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl</value>
+    <description>
+      Class to implement the State Store. By default it uses the local disk.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.federation.router.store.connection.test</name>
+    <value>60000</value>
+    <description>
+      How often to check for the connection to the State Store in milliseconds.
+    </description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/13f300e0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
new file mode 100644
index 0000000..fc5aebd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java
@@ -0,0 +1,232 @@
+/**
+ * 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.store;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS;
+import static org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileImpl.FEDERATION_STORE_FILE_DIRECTORY;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreFileBaseImpl;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.Time;
+
+/**
+ * Utilities to test the State Store.
+ */
+public final class FederationStateStoreTestUtils {
+
+  private FederationStateStoreTestUtils() {
+    // Utility Class
+  }
+
+  /**
+   * Get the default State Store driver implementation.
+   *
+   * @return Class of the default State Store driver implementation.
+   */
+  public static Class<? extends StateStoreDriver> getDefaultDriver() {
+    return DFSConfigKeys.FEDERATION_STORE_DRIVER_CLASS_DEFAULT;
+  }
+
+  /**
+   * Create a default State Store configuration.
+   *
+   * @return State Store configuration.
+   */
+  public static Configuration getStateStoreConfiguration() {
+    Class<? extends StateStoreDriver> clazz = getDefaultDriver();
+    return getStateStoreConfiguration(clazz);
+  }
+
+  /**
+   * Create a new State Store configuration for a particular driver.
+   *
+   * @param clazz Class of the driver to create.
+   * @return State Store configuration.
+   */
+  public static Configuration getStateStoreConfiguration(
+      Class<? extends StateStoreDriver> clazz) {
+    Configuration conf = new HdfsConfiguration(false);
+
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "hdfs://test");
+
+    conf.setClass(FEDERATION_STORE_DRIVER_CLASS, clazz, StateStoreDriver.class);
+
+    if (clazz.isAssignableFrom(StateStoreFileBaseImpl.class)) {
+      setFileConfiguration(conf);
+    }
+    return conf;
+  }
+
+  /**
+   * Create a new State Store based on a configuration.
+   *
+   * @param configuration Configuration for the State Store.
+   * @return New State Store service.
+   * @throws IOException If it cannot create the State Store.
+   * @throws InterruptedException If we cannot wait for the store to start.
+   */
+  public static StateStoreService getStateStore(
+      Configuration configuration) throws IOException, InterruptedException {
+
+    StateStoreService stateStore = new StateStoreService();
+    assertNotNull(stateStore);
+
+    // Set unique identifier, this is normally the router address
+    String identifier = UUID.randomUUID().toString();
+    stateStore.setIdentifier(identifier);
+
+    stateStore.init(configuration);
+    stateStore.start();
+
+    // Wait for state store to connect
+    waitStateStore(stateStore, TimeUnit.SECONDS.toMillis(10));
+
+    return stateStore;
+  }
+
+  /**
+   * Wait for the State Store to initialize its driver.
+   *
+   * @param stateStore State Store.
+   * @param timeoutMs Time out in milliseconds.
+   * @throws IOException If the State Store cannot be reached.
+   * @throws InterruptedException If the sleep is interrupted.
+   */
+  public static void waitStateStore(StateStoreService stateStore,
+      long timeoutMs) throws IOException, InterruptedException {
+    long startingTime = Time.monotonicNow();
+    while (!stateStore.isDriverReady()) {
+      Thread.sleep(100);
+      if (Time.monotonicNow() - startingTime > timeoutMs) {
+        throw new IOException("Timeout waiting for State Store to connect");
+      }
+    }
+  }
+
+  /**
+   * Delete the default State Store.
+   *
+   * @throws IOException
+   */
+  public static void deleteStateStore() throws IOException {
+    Class<? extends StateStoreDriver> driverClass = getDefaultDriver();
+    deleteStateStore(driverClass);
+  }
+
+  /**
+   * Delete the State Store.
+   * @param driverClass Class of the State Store driver implementation.
+   * @throws IOException If it cannot be deleted.
+   */
+  public static void deleteStateStore(
+      Class<? extends StateStoreDriver> driverClass) throws IOException {
+
+    if (StateStoreFileBaseImpl.class.isAssignableFrom(driverClass)) {
+      String workingDirectory = System.getProperty("user.dir");
+      File dir = new File(workingDirectory + "/statestore");
+      if (dir.exists()) {
+        FileUtils.cleanDirectory(dir);
+      }
+    }
+  }
+
+  /**
+   * Set the default configuration for drivers based on files.
+   *
+   * @param conf Configuration to extend.
+   */
+  public static void setFileConfiguration(Configuration conf) {
+    String workingPath = System.getProperty("user.dir");
+    String stateStorePath = workingPath + "/statestore";
+    conf.set(FEDERATION_STORE_FILE_DIRECTORY, stateStorePath);
+  }
+
+  /**
+   * Clear all the records from the State Store.
+   *
+   * @param store State Store to remove records from.
+   * @return If the State Store was cleared.
+   * @throws IOException If it cannot clear the State Store.
+   */
+  public static boolean clearAllRecords(StateStoreService store)
+      throws IOException {
+    Collection<Class<? extends BaseRecord>> allRecords =
+        store.getSupportedRecords();
+    for (Class<? extends BaseRecord> recordType : allRecords) {
+      if (!clearRecords(store, recordType)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Clear records from a certain type from the State Store.
+   *
+   * @param store State Store to remove records from.
+   * @param recordClass Class of the records to remove.
+   * @return If the State Store was cleared.
+   * @throws IOException If it cannot clear the State Store.
+   */
+  public static <T extends BaseRecord> boolean clearRecords(
+      StateStoreService store, Class<T> recordClass) throws IOException {
+    List<T> emptyList = new ArrayList<>();
+    if (!synchronizeRecords(store, emptyList, recordClass)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Synchronize a set of records. Remove all and keep the ones specified.
+   *
+   * @param stateStore State Store service managing the driver.
+   * @param records Records to add.
+   * @param clazz Class of the record to synchronize.
+   * @return If the synchronization succeeded.
+   * @throws IOException If it cannot connect to the State Store.
+   */
+  public static <T extends BaseRecord> boolean synchronizeRecords(
+      StateStoreService stateStore, List<T> records, Class<T> clazz)
+          throws IOException {
+    StateStoreDriver driver = stateStore.getDriver();
+    driver.verifyDriverReady();
+    if (driver.removeAll(clazz)) {
+      if (driver.putAll(records, true, false)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
\ 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


[39/50] [abbrv] hadoop git commit: YARN-6864. FSPreemptionThread cleanup for readability. (Daniel Templeton via Yufei Gu)

Posted by in...@apache.org.
YARN-6864. FSPreemptionThread cleanup for readability. (Daniel Templeton via Yufei Gu)


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

Branch: refs/heads/HDFS-10467
Commit: 9902be72cbf7a170caa5cb1f13c227d881a39064
Parents: 38c6fa5
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Jul 27 23:19:39 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Thu Jul 27 23:19:39 2017 -0700

----------------------------------------------------------------------
 .../scheduler/fair/FSPreemptionThread.java      | 28 ++++++++++----------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9902be72/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.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/fair/FSPreemptionThread.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
index efe36a6..b3e59c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java
@@ -66,11 +66,11 @@ class FSPreemptionThread extends Thread {
     schedulerReadLock = scheduler.getSchedulerReadLock();
   }
 
+  @Override
   public void run() {
     while (!Thread.interrupted()) {
-      FSAppAttempt starvedApp;
-      try{
-        starvedApp = context.getStarvedApps().take();
+      try {
+        FSAppAttempt starvedApp = context.getStarvedApps().take();
         // Hold the scheduler readlock so this is not concurrent with the
         // update thread.
         schedulerReadLock.lock();
@@ -82,7 +82,7 @@ class FSPreemptionThread extends Thread {
         starvedApp.preemptionTriggered(delayBeforeNextStarvationCheck);
       } catch (InterruptedException e) {
         LOG.info("Preemption thread interrupted! Exiting.");
-        return;
+        Thread.currentThread().interrupt();
       }
     }
   }
@@ -112,16 +112,19 @@ class FSPreemptionThread extends Thread {
         PreemptableContainers bestContainers = null;
         List<FSSchedulerNode> potentialNodes = scheduler.getNodeTracker()
             .getNodesByResourceName(rr.getResourceName());
+        int maxAMContainers = Integer.MAX_VALUE;
+
         for (FSSchedulerNode node : potentialNodes) {
-          int maxAMContainers = bestContainers == null ?
-              Integer.MAX_VALUE : bestContainers.numAMContainers;
           PreemptableContainers preemptableContainers =
               identifyContainersToPreemptOnNode(
                   rr.getCapability(), node, maxAMContainers);
+
           if (preemptableContainers != null) {
             // This set is better than any previously identified set.
             bestContainers = preemptableContainers;
-            if (preemptableContainers.numAMContainers == 0) {
+            maxAMContainers = bestContainers.numAMContainers;
+
+            if (maxAMContainers == 0) {
               break;
             }
           }
@@ -182,13 +185,10 @@ class FSPreemptionThread extends Thread {
         return preemptableContainers;
       }
     }
-    return null;
-  }
 
-  private boolean isNodeAlreadyReserved(
-      FSSchedulerNode node, FSAppAttempt app) {
-    FSAppAttempt nodeReservedApp = node.getReservedAppSchedulable();
-    return nodeReservedApp != null && !nodeReservedApp.equals(app);
+    // Return null if the sum of all preemptable containers' resources
+    // isn't enough to satisfy the starved request.
+    return null;
   }
 
   private void trackPreemptionsAgainstNode(List<RMContainer> containers,
@@ -214,7 +214,7 @@ class FSPreemptionThread extends Thread {
   }
 
   private class PreemptContainersTask extends TimerTask {
-    private List<RMContainer> containers;
+    private final List<RMContainer> containers;
 
     PreemptContainersTask(List<RMContainer> containers) {
       this.containers = containers;


---------------------------------------------------------------------
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: YARN-6844. AMRMClientImpl.checkNodeLabelExpression() has wrong error message (Contributed by Manikandan R via Daniel Templeton)

Posted by in...@apache.org.
YARN-6844. AMRMClientImpl.checkNodeLabelExpression() has wrong error message
(Contributed by Manikandan R via Daniel Templeton)


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

Branch: refs/heads/HDFS-10467
Commit: 4c40cd451cbdbce5d2b94ad0e7e3cc991c3439c5
Parents: 24853bf
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Jul 24 12:03:02 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Jul 24 12:03:02 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c40cd45/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index 7da91de..7a21bc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -772,10 +772,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
       return;
     }
 
-    // Don't support specifying >= 2 node labels in a node label expression now
+    // Don't support specifying > 1 node labels in a node label expression now
     if (exp.contains("&&") || exp.contains("||")) {
       throw new InvalidContainerRequestException(
-          "Cannot specify more than two node labels"
+          "Cannot specify more than one node label"
               + " in a single node label expression");
     }
   }


---------------------------------------------------------------------
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: HDFS-12171. Reduce IIP object allocations for inode lookup. Contributed by Daryn Sharp.

Posted by in...@apache.org.
HDFS-12171. Reduce IIP object allocations for inode lookup. Contributed by Daryn Sharp.


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

Branch: refs/heads/HDFS-10467
Commit: a68b5b31cf846c0fc94c430bafd07a9bca369234
Parents: 6d983cc
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Jul 25 11:03:09 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Jul 25 11:03:49 2017 -0500

----------------------------------------------------------------------
 .../server/namenode/EncryptionZoneManager.java   |  5 ++---
 .../server/namenode/FSDirErasureCodingOp.java    |  5 ++---
 .../hdfs/server/namenode/INodesInPath.java       | 19 ++-----------------
 .../server/namenode/TestSnapshotPathINodes.java  |  5 ++---
 4 files changed, 8 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a68b5b31/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index 6dff62b..96e189b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -209,9 +209,8 @@ public class EncryptionZoneManager {
     if (!hasCreatedEncryptionZone()) {
       return null;
     }
-    List<INode> inodes = iip.getReadOnlyINodes();
-    for (int i = inodes.size() - 1; i >= 0; i--) {
-      final INode inode = inodes.get(i);
+    for (int i = iip.length() - 1; i >= 0; i--) {
+      final INode inode = iip.getINode(i);
       if (inode != null) {
         final EncryptionZoneInt ezi = encryptionZones.get(inode.getId());
         if (ezi != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a68b5b31/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 681f217..486503c 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
@@ -355,9 +355,8 @@ final class FSDirErasureCodingOp {
     Preconditions.checkNotNull(iip, "INodes cannot be null");
     fsd.readLock();
     try {
-      List<INode> inodes = iip.getReadOnlyINodes();
-      for (int i = inodes.size() - 1; i >= 0; i--) {
-        final INode inode = inodes.get(i);
+      for (int i = iip.length() - 1; i >= 0; i--) {
+        final INode inode = iip.getINode(i);
         if (inode == null) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a68b5b31/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index abc8b63..8235bf0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -18,9 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.NoSuchElementException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -336,17 +333,9 @@ public class INodesInPath {
    *         otherwise, i < 0, return the (length + i)-th inode.
    */
   public INode getINode(int i) {
-    if (inodes == null || inodes.length == 0) {
-      throw new NoSuchElementException("inodes is null or empty");
-    }
-    int index = i >= 0 ? i : inodes.length + i;
-    if (index < inodes.length && index >= 0) {
-      return inodes[index];
-    } else {
-      throw new NoSuchElementException("inodes.length == " + inodes.length);
-    }
+    return inodes[(i < 0) ? inodes.length + i : i];
   }
-  
+
   /** @return the last inode. */
   public INode getLastINode() {
     return getINode(-1);
@@ -384,10 +373,6 @@ public class INodesInPath {
     return inodes.length;
   }
 
-  public List<INode> getReadOnlyINodes() {
-    return Collections.unmodifiableList(Arrays.asList(inodes));
-  }
-
   public INode[] getINodesArray() {
     INode[] retArr = new INode[inodes.length];
     System.arraycopy(inodes, 0, retArr, 0, inodes.length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a68b5b31/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index d1d915e..b62a418 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -310,9 +310,8 @@ public class TestSnapshotPathINodes {
   }
 
   private int getNumNonNull(INodesInPath iip) {
-    List<INode> inodes = iip.getReadOnlyINodes();
-    for (int i = inodes.size() - 1; i >= 0; i--) {
-      if (inodes.get(i) != null) {
+    for (int i = iip.length() - 1; i >= 0; i--) {
+      if (iip.getINode(i) != null) {
         return i+1;
       }
     }


---------------------------------------------------------------------
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: YARN-6779. DominantResourceFairnessPolicy.DominantResourceFairnessComparator.calculateShares() should be @VisibleForTesting (Contributed by Yeliang Cang via Daniel Templeton)

Posted by in...@apache.org.
YARN-6779. DominantResourceFairnessPolicy.DominantResourceFairnessComparator.calculateShares() should be @VisibleForTesting
(Contributed by Yeliang Cang via Daniel Templeton)


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

Branch: refs/heads/HDFS-10467
Commit: bb30bd3771442df253cbe55c448379580bd5ad07
Parents: 4c40cd4
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Jul 24 12:13:50 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Jul 24 12:13:50 2017 -0700

----------------------------------------------------------------------
 .../scheduler/fair/policies/DominantResourceFairnessPolicy.java    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb30bd37/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.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/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 193ed4d..72377b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies;
 import java.util.Collection;
 import java.util.Comparator;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -174,6 +175,7 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
      * by largest share.  So if resource=<10 MB, 5 CPU>, and pool=<100 MB, 10 CPU>,
      * shares will be [.1, .5] and resourceOrder will be [CPU, MEMORY].
      */
+    @VisibleForTesting
     void calculateShares(Resource resource, Resource pool,
         ResourceWeights shares, ResourceType[] resourceOrder, ResourceWeights weights) {
       shares.setWeight(MEMORY, (float)resource.getMemorySize() /


---------------------------------------------------------------------
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-11742. Improve balancer usability after HDFS-8818. Contributed by Kihwal Lee

Posted by in...@apache.org.
HDFS-11742. Improve balancer usability after HDFS-8818. Contributed by Kihwal Lee


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

Branch: refs/heads/HDFS-10467
Commit: 8e3a992eccff26a7344c3f0e719898fa97706b8c
Parents: 3b48f81
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Jul 21 09:14:19 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Jul 21 09:14:19 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hdfs/server/balancer/Dispatcher.java | 36 +++++++++++++++++++-
 1 file changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8e3a992e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index f855e45..9270fde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -49,6 +49,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -121,6 +122,7 @@ public class Dispatcher {
 
   /** The maximum number of concurrent blocks moves at a datanode */
   private final int maxConcurrentMovesPerNode;
+  private final int maxMoverThreads;
 
   private final long getBlocksSize;
   private final long getBlocksMinBlockSize;
@@ -139,11 +141,13 @@ public class Dispatcher {
   static class Allocator {
     private final int max;
     private int count = 0;
+    private int lotSize = 1;
 
     Allocator(int max) {
       this.max = max;
     }
 
+    /** Allocate specified number of items */
     synchronized int allocate(int n) {
       final int remaining = max - count;
       if (remaining <= 0) {
@@ -155,9 +159,19 @@ public class Dispatcher {
       }
     }
 
+    /** Aloocate a single lot of items */
+    int allocate() {
+      return allocate(lotSize);
+    }
+
     synchronized void reset() {
       count = 0;
     }
+
+    /** Set the lot size */
+    synchronized void setLotSize(int lotSize) {
+      this.lotSize = lotSize;
+    }
   }
 
   private static class GlobalBlockMap {
@@ -1017,6 +1031,7 @@ public class Dispatcher {
     this.dispatchExecutor = dispatcherThreads == 0? null
         : Executors.newFixedThreadPool(dispatcherThreads);
     this.moverThreadAllocator = new Allocator(moverThreads);
+    this.maxMoverThreads = moverThreads;
     this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
 
     this.getBlocksSize = getBlocksSize;
@@ -1116,7 +1131,7 @@ public class Dispatcher {
     final DDatanode targetDn = p.target.getDDatanode();
     ExecutorService moveExecutor = targetDn.getMoveExecutor();
     if (moveExecutor == null) {
-      final int nThreads = moverThreadAllocator.allocate(maxConcurrentMovesPerNode);
+      final int nThreads = moverThreadAllocator.allocate();
       if (nThreads > 0) {
         moveExecutor = targetDn.initMoveExecutor(nThreads);
       }
@@ -1166,6 +1181,25 @@ public class Dispatcher {
       LOG.debug("Disperse Interval sec = " +
           concurrentThreads / BALANCER_NUM_RPC_PER_SEC);
     }
+
+    // Determine the size of each mover thread pool per target
+    int threadsPerTarget = maxMoverThreads/targets.size();
+    if (threadsPerTarget == 0) {
+      // Some scheduled moves will get ignored as some targets won't have
+      // any threads allocated.
+      moverThreadAllocator.setLotSize(1);
+      LOG.warn(DFSConfigKeys.DFS_BALANCER_MOVERTHREADS_KEY + "=" +
+          maxMoverThreads + " is too small for moving blocks to " +
+          targets.size() + " targets. Balancing may be slower.");
+    } else {
+      if  (threadsPerTarget > maxConcurrentMovesPerNode) {
+        threadsPerTarget = maxConcurrentMovesPerNode;
+        LOG.info("Limiting threads per target to the specified max.");
+      }
+      moverThreadAllocator.setLotSize(threadsPerTarget);
+      LOG.info("Allocating " + threadsPerTarget + " threads per target.");
+    }
+
     long dSec = 0;
     final Iterator<Source> i = sources.iterator();
     for (int j = 0; j < futures.length; j++) {


---------------------------------------------------------------------
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: YARN-6150. TestContainerManagerSecurity tests for Yarn Server are flakey. Contributed by Daniel Sturman and Ray Chiang.

Posted by in...@apache.org.
YARN-6150. TestContainerManagerSecurity tests for Yarn Server are flakey. Contributed by Daniel Sturman and Ray Chiang.


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

Branch: refs/heads/HDFS-10467
Commit: 218b1b33ffe83cf2e330a2aa90685d0c14547a3d
Parents: f2921e5
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jul 25 15:11:21 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Jul 25 15:12:08 2017 +0900

----------------------------------------------------------------------
 .../server/TestContainerManagerSecurity.java    | 74 +++++++++++++-------
 1 file changed, 50 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/218b1b33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
index 9626b35..3ba4beb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
@@ -122,7 +123,13 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     testRootDir.delete();
   }
 
-  @Parameters
+  /*
+   * Run two tests: one with no security ("simple") and one with "Secure"
+   * The first parameter is just the test name to make it easier to debug
+   * and to give details in say an IDE.  The second is the configuraiton
+   * object to use.
+   */
+  @Parameters(name = "{0}")
   public static Collection<Object[]> configs() {
     Configuration configurationWithoutSecurity = new Configuration();
     configurationWithoutSecurity.set(
@@ -142,16 +149,18 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
       YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
       httpSpnegoKeytabFile.getAbsolutePath());
 
-    return Arrays.asList(new Object[][] { { configurationWithoutSecurity },
-        { configurationWithSecurity } });
+    return Arrays.asList(new Object[][] {
+        {"Simple", configurationWithoutSecurity},
+        {"Secure", configurationWithSecurity}});
   }
   
-  public TestContainerManagerSecurity(Configuration conf) {
+  public TestContainerManagerSecurity(String name, Configuration conf) {
+    LOG.info("RUNNING TEST " + name);
     conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 100000L);
     this.conf = conf;
   }
   
-  @Test (timeout = 120000)
+  @Test
   public void testContainerManager() throws Exception {
       
       // TestNMTokens.
@@ -165,7 +174,11 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
 
   }
 
-  private void testNMTokens(Configuration conf) throws Exception {
+  /**
+   * Run a series of tests using different NMTokens.  A configuration is
+   * provided for managing creating of the tokens and rpc.
+   */
+  private void testNMTokens(Configuration testConf) throws Exception {
     NMTokenSecretManagerInRM nmTokenSecretManagerRM =
         yarnCluster.getResourceManager().getRMContext()
           .getNMTokenSecretManager();
@@ -201,7 +214,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
      * nmToken error. (This can be best tested if we roll over NMToken
      * master key twice).
      */
-    YarnRPC rpc = YarnRPC.create(conf);
+    YarnRPC rpc = YarnRPC.create(testConf);
     String user = "test";
     Resource r = Resource.newInstance(1024, 1);
 
@@ -233,7 +246,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     StringBuilder sb;
     // testInvalidNMToken ... creating NMToken using different secret manager.
     
-    NMTokenSecretManagerInRM tempManager = new NMTokenSecretManagerInRM(conf);
+    NMTokenSecretManagerInRM tempManager = new NMTokenSecretManagerInRM(testConf);
     tempManager.rollMasterKey();
     do {
       tempManager.rollMasterKey();
@@ -252,7 +265,9 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     }
     String errorMsg = testStartContainer(rpc, validAppAttemptId, validNode,
         validContainerToken, null, true);
-    Assert.assertTrue(errorMsg.contains(sb.toString()));
+    Assert.assertTrue("In calling " + validNode + " exception was '"
+        + errorMsg + "' but doesn't contain '"
+        + sb.toString() + "'", errorMsg.contains(sb.toString()));
     
     org.apache.hadoop.yarn.api.records.Token invalidNMToken =
         tempManager.createNMToken(validAppAttemptId, validNode, user);
@@ -277,7 +292,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
             validContainerToken, invalidNMToken, true)));
     
     // using correct tokens. nmtoken for app attempt should get saved.
-    conf.setInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
+    testConf.setInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
         4 * 60 * 1000);
     validContainerToken =
         containerTokenSecretManager.createContainerToken(validContainerId,
@@ -375,8 +390,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     Assert.assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
         validContainerId, validNMToken, false).contains(sb.toString()));
 
-    // using appAttempt-1 NMtoken for launching container for appAttempt-2 should
-    // succeed.
+    // using appAttempt-1 NMtoken for launching container for appAttempt-2
+    // should succeed.
     ApplicationAttemptId attempt2 = ApplicationAttemptId.newInstance(appId, 2);
     Token attempt1NMToken =
         nmTokenSecretManagerRM
@@ -390,13 +405,20 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
   }
 
   private void waitForContainerToFinishOnNM(ContainerId containerId) {
-    Context nmContet = yarnCluster.getNodeManager(0).getNMContext();
+    Context nmContext = yarnCluster.getNodeManager(0).getNMContext();
     int interval = 4 * 60; // Max time for container token to expire.
-    Assert.assertNotNull(nmContet.getContainers().containsKey(containerId));
+
+    Assert.assertNotNull(nmContext.getContainers().containsKey(containerId));
+
+    // Get the container first, as it may be removed from the Context
+    // by asynchronous calls.
+    // This was leading to a flakey test as otherwise the container could
+    // be removed and end up null.
+    Container waitContainer = nmContext.getContainers().get(containerId);
+
     while ((interval-- > 0)
-        && !nmContet.getContainers().get(containerId)
-          .cloneAndGetContainerStatus().getState()
-          .equals(ContainerState.COMPLETE)) {
+        && !waitContainer.cloneAndGetContainerStatus()
+        .getState().equals(ContainerState.COMPLETE)) {
       try {
         LOG.info("Waiting for " + containerId + " to complete.");
         Thread.sleep(1000);
@@ -407,7 +429,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     // explicitly acked by RM. Now, manually remove it for testing.
     yarnCluster.getNodeManager(0).getNodeStatusUpdater()
       .addCompletedContainer(containerId);
-    nmContet.getContainers().remove(containerId);
+    LOG.info("Removing container from NMContext, containerID = " + containerId);
+    nmContext.getContainers().remove(containerId);
   }
 
   protected void waitForNMToReceiveNMTokenKey(
@@ -439,7 +462,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
       ContainerId containerId, Token nmToken, boolean isExceptionExpected) {
     try {
       stopContainer(rpc, nmToken,
-          Arrays.asList(new ContainerId[] { containerId }), appAttemptId,
+          Arrays.asList(new ContainerId[] {containerId}), appAttemptId,
           nodeId);
       if (isExceptionExpected) {
         fail("Exception was expected!!");
@@ -525,7 +548,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
       proxy =
           getContainerManagementProtocolProxy(rpc, nmToken, nodeId,
               appAttemptId.toString());
-      GetContainerStatusesResponse statuses = proxy.getContainerStatuses(request);
+      GetContainerStatusesResponse statuses
+          = proxy.getContainerStatuses(request);
       if (statuses.getFailedRequests() != null
           && statuses.getFailedRequests().containsKey(containerId)) {
         parseAndThrowException(statuses.getFailedRequests().get(containerId)
@@ -546,7 +570,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ContainerLaunchContext context =
         Records.newRecord(ContainerLaunchContext.class);
     StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(context,containerToken);
+        StartContainerRequest.newInstance(context, containerToken);
     List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
     list.add(scRequest);
     StartContainersRequest allRequests =
@@ -582,7 +606,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ContainerManagementProtocol proxy;
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
     final InetSocketAddress addr =
-        NetUtils.createSocketAddr(nodeId.getHost(), nodeId.getPort());
+        new InetSocketAddress(nodeId.getHost(), nodeId.getPort());
     if (nmToken != null) {
       ugi.addToken(ConverterUtils.convertFromYarn(nmToken, addr));      
     }
@@ -645,9 +669,11 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ContainerTokenIdentifier containerTokenIdentifier = 
         getContainerTokenIdentifierFromToken(containerToken);
     
-    // Verify new compatible version ContainerTokenIdentifier can work successfully.
+    // Verify new compatible version ContainerTokenIdentifier
+    // can work successfully.
     ContainerTokenIdentifierForTest newVersionTokenIdentifier = 
-        new ContainerTokenIdentifierForTest(containerTokenIdentifier, "message");
+        new ContainerTokenIdentifierForTest(containerTokenIdentifier,
+            "message");
     byte[] password = 
         containerTokenSecretManager.createPassword(newVersionTokenIdentifier);
     


---------------------------------------------------------------------
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-12143. Improve performance of getting and removing inode features. Contributed by Daryn Sharp.

Posted by in...@apache.org.
HDFS-12143. Improve performance of getting and removing inode features. Contributed by Daryn Sharp.


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

Branch: refs/heads/HDFS-10467
Commit: 1a79dcfc457969d6a6c08ffffe4152fd7638e48a
Parents: cca51e9
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Jul 25 10:28:57 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Tue Jul 25 10:28:57 2017 -0500

----------------------------------------------------------------------
 .../namenode/INodeWithAdditionalFields.java     | 24 ++++++++++++++------
 1 file changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a79dcfc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
index fe58577..9adcc3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
@@ -283,12 +283,14 @@ public abstract class INodeWithAdditionalFields extends INode
 
   protected void removeFeature(Feature f) {
     int size = features.length;
-    Preconditions.checkState(size > 0, "Feature "
-        + f.getClass().getSimpleName() + " not found.");
+    if (size == 0) {
+      throwFeatureNotFoundException(f);
+    }
 
     if (size == 1) {
-      Preconditions.checkState(features[0] == f, "Feature "
-          + f.getClass().getSimpleName() + " not found.");
+      if (features[0] != f) {
+        throwFeatureNotFoundException(f);
+      }
       features = EMPTY_FEATURE;
       return;
     }
@@ -307,14 +309,22 @@ public abstract class INodeWithAdditionalFields extends INode
       }
     }
 
-    Preconditions.checkState(!overflow && j == size - 1, "Feature "
-        + f.getClass().getSimpleName() + " not found.");
+    if (overflow || j != size - 1) {
+      throwFeatureNotFoundException(f);
+    }
     features = arr;
   }
 
+  private void throwFeatureNotFoundException(Feature f) {
+    throw new IllegalStateException(
+        "Feature " + f.getClass().getSimpleName() + " not found.");
+  }
+
   protected <T extends Feature> T getFeature(Class<? extends Feature> clazz) {
     Preconditions.checkArgument(clazz != null);
-    for (Feature f : features) {
+    final int size = features.length;
+    for (int i=0; i < size; i++) {
+      Feature f = features[i];
       if (clazz.isAssignableFrom(f.getClass())) {
         @SuppressWarnings("unchecked")
         T ret = (T) f;


---------------------------------------------------------------------
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: HDFS-10882. Federation State Store Interface API. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
HDFS-10882. Federation State Store Interface API. Contributed by Jason Kace and Inigo Goiri.


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

Branch: refs/heads/HDFS-10467
Commit: 1e50322c6dc5f6323452a9190ca48b106f6c7a10
Parents: b4b7b15
Author: Inigo <in...@apache.org>
Authored: Thu Apr 6 19:18:52 2017 -0700
Committer: Inigo Goiri <in...@apache.org>
Committed: Fri Jul 28 09:48:38 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  11 ++
 .../server/federation/store/RecordStore.java    | 100 ++++++++++++++++
 .../store/driver/StateStoreSerializer.java      | 119 +++++++++++++++++++
 .../driver/impl/StateStoreSerializerPBImpl.java | 115 ++++++++++++++++++
 .../store/records/impl/pb/PBRecord.java         |  47 ++++++++
 .../store/records/impl/pb/package-info.java     |  29 +++++
 .../src/main/resources/hdfs-default.xml         |   8 ++
 7 files changed, 429 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 71f999a..cd7f1a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
+import org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl;
 import org.apache.hadoop.http.HttpConfig;
 
 /** 
@@ -1104,6 +1105,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS_DEFAULT =
       "org.apache.hadoop.hdfs.server.federation.MockResolver";
 
+  // HDFS Router-based federation State Store
+  public static final String FEDERATION_STORE_PREFIX =
+      FEDERATION_ROUTER_PREFIX + "store.";
+
+  public static final String FEDERATION_STORE_SERIALIZER_CLASS =
+      DFSConfigKeys.FEDERATION_STORE_PREFIX + "serializer";
+  public static final Class<StateStoreSerializerPBImpl>
+      FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT =
+          StateStoreSerializerPBImpl.class;
+
   // dfs.client.retry confs are moved to HdfsClientConfigKeys.Retry 
   @Deprecated
   public static final String  DFS_CLIENT_RETRY_POLICY_ENABLED_KEY

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
new file mode 100644
index 0000000..524f432
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
@@ -0,0 +1,100 @@
+/**
+ * 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.store;
+
+import java.lang.reflect.Constructor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+
+/**
+ * Store records in the State Store. Subclasses provide interfaces to operate on
+ * those records.
+ *
+ * @param <R> Record to store by this interface.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class RecordStore<R extends BaseRecord> {
+
+  private static final Log LOG = LogFactory.getLog(RecordStore.class);
+
+
+  /** Class of the record stored in this State Store. */
+  private final Class<R> recordClass;
+
+  /** State store driver backed by persistent storage. */
+  private final StateStoreDriver driver;
+
+
+  /**
+   * Create a new store for records.
+   *
+   * @param clazz Class of the record to store.
+   * @param stateStoreDriver Driver for the State Store.
+   */
+  protected RecordStore(Class<R> clazz, StateStoreDriver stateStoreDriver) {
+    this.recordClass = clazz;
+    this.driver = stateStoreDriver;
+  }
+
+  /**
+   * Report a required record to the data store. The data store uses this to
+   * create/maintain storage for the record.
+   *
+   * @return The class of the required record or null if no record is required
+   *         for this interface.
+   */
+  public Class<R> getRecordClass() {
+    return this.recordClass;
+  }
+
+  /**
+   * Get the State Store driver.
+   *
+   * @return State Store driver.
+   */
+  protected StateStoreDriver getDriver() {
+    return this.driver;
+  }
+
+  /**
+   * Build a state store API implementation interface.
+   *
+   * @param interfaceClass The specific interface implementation to create
+   * @param driver The {@link StateStoreDriver} implementation in use.
+   * @return An initialized instance of the specified state store API
+   *         implementation.
+   */
+  public static <T extends RecordStore<?>> T newInstance(
+      final Class<T> clazz, final StateStoreDriver driver) {
+
+    try {
+      Constructor<T> constructor = clazz.getConstructor(StateStoreDriver.class);
+      T recordStore = constructor.newInstance(driver);
+      return recordStore;
+    } catch (Exception e) {
+      LOG.error("Cannot create new instance for " + clazz, e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
new file mode 100644
index 0000000..8540405
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
@@ -0,0 +1,119 @@
+/**
+ * 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.store.driver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Serializer to store and retrieve data in the State Store.
+ */
+public abstract class StateStoreSerializer {
+
+  /** Singleton for the serializer instance. */
+  private static StateStoreSerializer defaultSerializer;
+
+  /**
+   * Get the default serializer based.
+   * @return Singleton serializer.
+   */
+  public static StateStoreSerializer getSerializer() {
+    return getSerializer(null);
+  }
+
+  /**
+   * Get a serializer based on the provided configuration.
+   * @param conf Configuration. Default if null.
+   * @return Singleton serializer.
+   */
+  public static StateStoreSerializer getSerializer(Configuration conf) {
+    if (conf == null) {
+      synchronized (StateStoreSerializer.class) {
+        if (defaultSerializer == null) {
+          conf = new Configuration();
+          defaultSerializer = newSerializer(conf);
+        }
+      }
+      return defaultSerializer;
+    } else {
+      return newSerializer(conf);
+    }
+  }
+
+  private static StateStoreSerializer newSerializer(final Configuration conf) {
+    Class<? extends StateStoreSerializer> serializerName = conf.getClass(
+        DFSConfigKeys.FEDERATION_STORE_SERIALIZER_CLASS,
+        DFSConfigKeys.FEDERATION_STORE_SERIALIZER_CLASS_DEFAULT,
+        StateStoreSerializer.class);
+    return ReflectionUtils.newInstance(serializerName, conf);
+  }
+
+  /**
+   * Create a new record.
+   * @param clazz Class of the new record.
+   * @return New record.
+   */
+  public static <T> T newRecord(Class<T> clazz) {
+    return getSerializer(null).newRecordInstance(clazz);
+  }
+
+  /**
+   * Create a new record.
+   * @param clazz Class of the new record.
+   * @return New record.
+   */
+  public abstract <T> T newRecordInstance(Class<T> clazz);
+
+  /**
+   * Serialize a record into a byte array.
+   * @param record Record to serialize.
+   * @return Byte array with the serialized record.
+   */
+  public abstract byte[] serialize(BaseRecord record);
+
+  /**
+   * Serialize a record into a string.
+   * @param record Record to serialize.
+   * @return String with the serialized record.
+   */
+  public abstract String serializeString(BaseRecord record);
+
+  /**
+   * Deserialize a bytes array into a record.
+   * @param byteArray Byte array to deserialize.
+   * @param clazz Class of the record.
+   * @return New record.
+   * @throws IOException If it cannot deserialize the record.
+   */
+  public abstract <T extends BaseRecord> T deserialize(
+      byte[] byteArray, Class<T> clazz) throws IOException;
+
+  /**
+   * Deserialize a string into a record.
+   * @param data String with the data to deserialize.
+   * @param clazz Class of the record.
+   * @return New record.
+   * @throws IOException If it cannot deserialize the record.
+   */
+  public abstract <T extends BaseRecord> T deserialize(
+      String data, Class<T> clazz) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java
new file mode 100644
index 0000000..45c5dd6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializerPBImpl.java
@@ -0,0 +1,115 @@
+/**
+ * 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.store.driver.impl;
+
+import java.io.IOException;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the State Store serializer.
+ */
+public final class StateStoreSerializerPBImpl extends StateStoreSerializer {
+
+  private static final String PB_IMPL_PACKAGE_SUFFIX = "impl.pb";
+  private static final String PB_IMPL_CLASS_SUFFIX = "PBImpl";
+
+  private Configuration localConf = new Configuration();
+
+
+  private StateStoreSerializerPBImpl() {
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T> T newRecordInstance(Class<T> clazz) {
+    try {
+      String clazzPBImpl = getPBImplClassName(clazz);
+      Class<?> pbClazz = localConf.getClassByName(clazzPBImpl);
+      Object retObject = ReflectionUtils.newInstance(pbClazz, localConf);
+      return (T)retObject;
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private String getPBImplClassName(Class<?> clazz) {
+    String srcPackagePart = getPackageName(clazz);
+    String srcClassName = getClassName(clazz);
+    String destPackagePart = srcPackagePart + "." + PB_IMPL_PACKAGE_SUFFIX;
+    String destClassPart = srcClassName + PB_IMPL_CLASS_SUFFIX;
+    return destPackagePart + "." + destClassPart;
+  }
+
+  private String getClassName(Class<?> clazz) {
+    String fqName = clazz.getName();
+    return (fqName.substring(fqName.lastIndexOf(".") + 1, fqName.length()));
+  }
+
+  private String getPackageName(Class<?> clazz) {
+    return clazz.getPackage().getName();
+  }
+
+  @Override
+  public byte[] serialize(BaseRecord record) {
+    byte[] byteArray64 = null;
+    if (record instanceof PBRecord) {
+      PBRecord recordPB = (PBRecord) record;
+      Message msg = recordPB.getProto();
+      byte[] byteArray = msg.toByteArray();
+      byteArray64 = Base64.encodeBase64(byteArray, false);
+    }
+    return byteArray64;
+  }
+
+  @Override
+  public String serializeString(BaseRecord record) {
+    byte[] byteArray64 = serialize(record);
+    String base64Encoded = StringUtils.newStringUtf8(byteArray64);
+    return base64Encoded;
+  }
+
+  @Override
+  public <T extends BaseRecord> T deserialize(
+      byte[] byteArray, Class<T> clazz) throws IOException {
+
+    T record = newRecord(clazz);
+    if (record instanceof PBRecord) {
+      PBRecord pbRecord = (PBRecord) record;
+      byte[] byteArray64 = Base64.encodeBase64(byteArray, false);
+      String base64Encoded = StringUtils.newStringUtf8(byteArray64);
+      pbRecord.readInstance(base64Encoded);
+    }
+    return record;
+  }
+
+  @Override
+  public <T extends BaseRecord> T deserialize(String data, Class<T> clazz)
+      throws IOException {
+    byte[] byteArray64 = Base64.decodeBase64(data);
+    return deserialize(byteArray64, clazz);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
new file mode 100644
index 0000000..c369275
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/PBRecord.java
@@ -0,0 +1,47 @@
+/**
+ * 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.store.records.impl.pb;
+
+import java.io.IOException;
+
+import com.google.protobuf.Message;
+
+/**
+ * A record implementation using Protobuf.
+ */
+public interface PBRecord {
+
+  /**
+   * Get the protocol for the record.
+   * @return The protocol for this record.
+   */
+  Message getProto();
+
+  /**
+   * Set the protocol for the record.
+   * @param proto Protocol for this record.
+   */
+  void setProto(Message proto);
+
+  /**
+   * Populate this record with serialized data.
+   * @param base64String Serialized data in base64.
+   * @throws IOException If it cannot read the data.
+   */
+  void readInstance(String base64String) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java
new file mode 100644
index 0000000..b329732
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/package-info.java
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+/**
+ * The protobuf implementations of state store data records defined in the
+ * org.apache.hadoop.hdfs.server.federation.store.records package. Each
+ * implementation wraps an associated protobuf proto definition.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.hdfs.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e50322c/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 4a4dc11..39fbf98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4559,4 +4559,12 @@
     </description>
   </property>
 
+  <property>
+    <name>dfs.federation.router.store.serializer</name>
+    <value>org.apache.hadoop.hdfs.server.federation.store.driver.impl.StateStoreSerializerPBImpl</value>
+    <description>
+      Class to serialize State Store records.
+    </description>
+  </property>
+
 </configuration>


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMErrorsAndWarningsPage.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/webapp/NMErrorsAndWarningsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMErrorsAndWarningsPage.java
index 7475c4d..5e81ed5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMErrorsAndWarningsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMErrorsAndWarningsPage.java
@@ -32,7 +32,7 @@ public class NMErrorsAndWarningsPage extends NMView {
   }
 
   @Override
-  protected void preHead(HtmlPage.Page.HTML<HtmlPage._> html) {
+  protected void preHead(HtmlPage.Page.HTML<__> html) {
     commonPreHead(html);
     String title = "Errors and Warnings in the NodeManager";
     setTitle(title);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMView.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/webapp/NMView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMView.java
index dc21b4a..a76d2ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NMView.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
 
 public class NMView extends TwoColumnLayout {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
       commonPreHead(html);
     }
 
-  protected void commonPreHead(Page.HTML<_> html) {
+  protected void commonPreHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.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/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java
index 857a4f9..0a2731e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java
@@ -24,7 +24,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
@@ -55,29 +55,29 @@ public class NavBlock extends HtmlBlock implements YarnWebParams {
         WebAppUtils.getResolvedRMWebAppURLWithScheme(this.conf);
 	  Hamlet.UL<Hamlet.DIV<Hamlet>> ul = html
       .div("#nav")
-      .h3()._("ResourceManager")._()
+      .h3().__("ResourceManager").__()
         .ul()
-          .li().a(RMWebAppURL, "RM Home")._()._()
-      .h3()._("NodeManager")._() // TODO: Problem if no header like this
+          .li().a(RMWebAppURL, "RM Home").__().__()
+      .h3().__("NodeManager").__() // TODO: Problem if no header like this
         .ul()
           .li()
-            .a(url("node"), "Node Information")._()
+            .a(url("node"), "Node Information").__()
           .li()
             .a(url("allApplications"), "List of Applications")
-            ._()
+            .__()
           .li()
-            .a(url("allContainers"), "List of Containers")._()
-        ._()
+            .a(url("allContainers"), "List of Containers").__()
+        .__()
       .h3("Tools")
         .ul()
-          .li().a("/conf", "Configuration")._()
-          .li().a("/logs", "Local logs")._()
-          .li().a("/stacks", "Server stacks")._()
-          .li().a("/jmx?qry=Hadoop:*", "Server metrics")._();
+          .li().a("/conf", "Configuration").__()
+          .li().a("/logs", "Local logs").__()
+          .li().a("/stacks", "Server stacks").__()
+          .li().a("/jmx?qry=Hadoop:*", "Server metrics").__();
     if (addErrorsAndWarningsLink) {
-      ul.li().a(url("errors-and-warnings"), "Errors/Warnings")._();
+      ul.li().a(url("errors-and-warnings"), "Errors/Warnings").__();
     }
-    ul._()._();
+    ul.__().__();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.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/webapp/NodePage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
index f51f0c5..7005f41 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NodePage.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.dao.NodeInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.HTML;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -39,7 +39,7 @@ public class NodePage extends NMView {
   private static final long BYTES_IN_MB = 1024 * 1024;
 
   @Override
-  protected void commonPreHead(HTML<_> html) {
+  protected void commonPreHead(HTML<__> html) {
     super.commonPreHead(html);
 
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
@@ -65,29 +65,29 @@ public class NodePage extends NMView {
     protected void render(Block html) {
       NodeInfo info = new NodeInfo(this.context, this.resourceView);
       info("NodeManager information")
-          ._("Total Vmem allocated for Containers",
+          .__("Total Vmem allocated for Containers",
               StringUtils.byteDesc(info.getTotalVmemAllocated() * BYTES_IN_MB))
-          ._("Vmem enforcement enabled",
+          .__("Vmem enforcement enabled",
               info.isVmemCheckEnabled())
-          ._("Total Pmem allocated for Container",
+          .__("Total Pmem allocated for Container",
               StringUtils.byteDesc(info.getTotalPmemAllocated() * BYTES_IN_MB))
-          ._("Pmem enforcement enabled",
+          .__("Pmem enforcement enabled",
               info.isPmemCheckEnabled())
-          ._("Total VCores allocated for Containers",
+          .__("Total VCores allocated for Containers",
               String.valueOf(info.getTotalVCoresAllocated()))
-          ._("NodeHealthyStatus",
+          .__("NodeHealthyStatus",
               info.getHealthStatus())
-          ._("LastNodeHealthTime", new Date(
+          .__("LastNodeHealthTime", new Date(
               info.getLastNodeUpdateTime()))
-          ._("NodeHealthReport",
+          .__("NodeHealthReport",
               info.getHealthReport())
-          ._("NodeManager started on", new Date(
+          .__("NodeManager started on", new Date(
               info.getNMStartupTime()))
-          ._("NodeManager Version:", info.getNMBuildVersion() +
+          .__("NodeManager Version:", info.getNMBuildVersion() +
               " on " + info.getNMVersionBuiltOn())
-          ._("Hadoop Version:", info.getHadoopBuildVersion() +
+          .__("Hadoop Version:", info.getHadoopBuildVersion() +
               " on " + info.getHadoopVersionBuiltOn());
-      html._(InfoBlock.class);
+      html.__(InfoBlock.class);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.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/AboutBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java
index 4225afd..b7ce105 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java
@@ -37,22 +37,22 @@ public class AboutBlock extends HtmlBlock {
 
   @Override
   protected void render(Block html) {
-    html._(MetricsOverviewTable.class);
+    html.__(MetricsOverviewTable.class);
     ResourceManager rm = getInstance(ResourceManager.class);
     ClusterInfo cinfo = new ClusterInfo(rm);
     info("Cluster overview").
-      _("Cluster ID:", cinfo.getClusterId()).
-      _("ResourceManager state:", cinfo.getState()).
-      _("ResourceManager HA state:", cinfo.getHAState()).
-      _("ResourceManager HA zookeeper connection state:",
+        __("Cluster ID:", cinfo.getClusterId()).
+        __("ResourceManager state:", cinfo.getState()).
+        __("ResourceManager HA state:", cinfo.getHAState()).
+        __("ResourceManager HA zookeeper connection state:",
           cinfo.getHAZookeeperConnectionState()).
-      _("ResourceManager RMStateStore:", cinfo.getRMStateStore()).
-      _("ResourceManager started on:", Times.format(cinfo.getStartedOn())).
-      _("ResourceManager version:", cinfo.getRMBuildVersion() +
+        __("ResourceManager RMStateStore:", cinfo.getRMStateStore()).
+        __("ResourceManager started on:", Times.format(cinfo.getStartedOn())).
+        __("ResourceManager version:", cinfo.getRMBuildVersion() +
           " on " + cinfo.getRMVersionBuiltOn()).
-      _("Hadoop version:", cinfo.getHadoopBuildVersion() +
+        __("Hadoop version:", cinfo.getHadoopBuildVersion() +
           " on " + cinfo.getHadoopVersionBuiltOn());
-    html._(InfoBlock.class);
+    html.__(InfoBlock.class);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutPage.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/AboutPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutPage.java
index ef0fdcf..f8c0406 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutPage.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.webapp.SubView;
 
 public class AboutPage extends RmView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.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/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
index 45f1887..89e2dec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AppAttemptPage extends RmView {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String appAttemptId = $(YarnWebParams.APPLICATION_ATTEMPT_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppLogAggregationStatusPage.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/AppLogAggregationStatusPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppLogAggregationStatusPage.java
index ccb53dd..27fb43a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppLogAggregationStatusPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppLogAggregationStatusPage.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AppLogAggregationStatusPage extends RmView{
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String appId = $(YarnWebParams.APPLICATION_ID);
     set(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.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/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
index 0c5516a..7036f33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class AppPage extends RmView {
 
   @Override 
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String appId = $(YarnWebParams.APPLICATION_ID);
     set(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.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/AppsBlockWithMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
index 29889ec..fac100f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
-import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 /**
@@ -26,7 +25,7 @@ import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
  */
 class AppsBlockWithMetrics extends HtmlBlock {
   @Override public void render(Block html) {
-    html._(MetricsOverviewTable.class);
-    html._(RMAppsBlock.class);
+    html.__(MetricsOverviewTable.class);
+    html.__(RMAppsBlock.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.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/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 292c5f3..f3ab5b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -47,12 +47,12 @@ import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.LI;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.UL;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -105,7 +105,7 @@ class CapacitySchedulerPage extends RmView {
           info("\'" + lqinfo.getQueuePath().substring(5)
               + "\' Queue Status for Partition \'" + nodeLabelDisplay + "\'");
       renderQueueCapacityInfo(ri, nodeLabel);
-      html._(InfoBlock.class);
+      html.__(InfoBlock.class);
       // clear the info contents so this queue's info doesn't accumulate into
       // another queue's info
       ri.clear();
@@ -113,10 +113,10 @@ class CapacitySchedulerPage extends RmView {
       // second display the queue specific details :
       ri =
           info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
-              ._("Queue State:", lqinfo.getQueueState());
+              .__("Queue State:", lqinfo.getQueueState());
       renderCommonLeafQueueInfo(ri);
 
-      html._(InfoBlock.class);
+      html.__(InfoBlock.class);
       // clear the info contents so this queue's info doesn't accumulate into
       // another queue's info
       ri.clear();
@@ -125,10 +125,10 @@ class CapacitySchedulerPage extends RmView {
     private void renderLeafQueueInfoWithoutParition(Block html) {
       ResponseInfo ri =
           info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
-              ._("Queue State:", lqinfo.getQueueState());
+              .__("Queue State:", lqinfo.getQueueState());
       renderQueueCapacityInfo(ri, "");
       renderCommonLeafQueueInfo(ri);
-      html._(InfoBlock.class);
+      html.__(InfoBlock.class);
       // clear the info contents so this queue's info doesn't accumulate into
       // another queue's info
       ri.clear();
@@ -155,40 +155,40 @@ class CapacitySchedulerPage extends RmView {
           ? new ResourceInfo(Resources.none())
           : resourceUsages.getAmUsed();
       ri.
-      _("Used Capacity:", percent(capacities.getUsedCapacity() / 100)).
-      _("Configured Capacity:", percent(capacities.getCapacity() / 100)).
-      _("Configured Max Capacity:", percent(capacities.getMaxCapacity() / 100)).
-      _("Absolute Used Capacity:", percent(capacities.getAbsoluteUsedCapacity() / 100)).
-      _("Absolute Configured Capacity:", percent(capacities.getAbsoluteCapacity() / 100)).
-      _("Absolute Configured Max Capacity:", percent(capacities.getAbsoluteMaxCapacity() / 100)).
-      _("Used Resources:", resourceUsages.getUsed().toString()).
-      _("Configured Max Application Master Limit:", StringUtils.format("%.1f",
+          __("Used Capacity:", percent(capacities.getUsedCapacity() / 100)).
+          __("Configured Capacity:", percent(capacities.getCapacity() / 100)).
+          __("Configured Max Capacity:", percent(capacities.getMaxCapacity() / 100)).
+          __("Absolute Used Capacity:", percent(capacities.getAbsoluteUsedCapacity() / 100)).
+          __("Absolute Configured Capacity:", percent(capacities.getAbsoluteCapacity() / 100)).
+          __("Absolute Configured Max Capacity:", percent(capacities.getAbsoluteMaxCapacity() / 100)).
+          __("Used Resources:", resourceUsages.getUsed().toString()).
+          __("Configured Max Application Master Limit:", StringUtils.format("%.1f",
           capacities.getMaxAMLimitPercentage())).
-      _("Max Application Master Resources:",
+          __("Max Application Master Resources:",
           resourceUsages.getAMLimit().toString()).
-      _("Used Application Master Resources:",
+          __("Used Application Master Resources:",
           amUsed.toString()).
-      _("Max Application Master Resources Per User:",
+          __("Max Application Master Resources Per User:",
           userAMResourceLimit.toString());
     }
 
     private void renderCommonLeafQueueInfo(ResponseInfo ri) {
       ri.
-      _("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
-      _("Num Non-Schedulable Applications:", Integer.toString(lqinfo.getNumPendingApplications())).
-      _("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
-      _("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
-      _("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
-      _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
-      _("Configured User Limit Factor:", lqinfo.getUserLimitFactor()).
-      _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
-      _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
-      _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled").
-      _("Default Node Label Expression:",
+          __("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
+          __("Num Non-Schedulable Applications:", Integer.toString(lqinfo.getNumPendingApplications())).
+          __("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
+          __("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
+          __("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
+          __("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
+          __("Configured User Limit Factor:", lqinfo.getUserLimitFactor()).
+          __("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
+          __("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
+          __("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled").
+          __("Default Node Label Expression:",
               lqinfo.getDefaultNodeLabelExpression() == null
                   ? NodeLabel.DEFAULT_NODE_LABEL_PARTITION
                   : lqinfo.getDefaultNodeLabelExpression()).
-      _("Default Application Priority:",
+          __("Default Application Priority:",
               Integer.toString(lqinfo.getDefaultApplicationPriority()));
     }
   }
@@ -208,14 +208,14 @@ class CapacitySchedulerPage extends RmView {
     protected void render(Block html) {
       TBODY<TABLE<Hamlet>> tbody =
           html.table("#userinfo").thead().$class("ui-widget-header").tr().th()
-              .$class("ui-state-default")._("User Name")._().th()
-              .$class("ui-state-default")._("Max Resource")._().th()
-              .$class("ui-state-default")._("Weight")._().th()
-              .$class("ui-state-default")._("Used Resource")._().th()
-              .$class("ui-state-default")._("Max AM Resource")._().th()
-              .$class("ui-state-default")._("Used AM Resource")._().th()
-              .$class("ui-state-default")._("Schedulable Apps")._().th()
-              .$class("ui-state-default")._("Non-Schedulable Apps")._()._()._()
+              .$class("ui-state-default").__("User Name").__().th()
+              .$class("ui-state-default").__("Max Resource").__().th()
+              .$class("ui-state-default").__("Weight").__().th()
+              .$class("ui-state-default").__("Used Resource").__().th()
+              .$class("ui-state-default").__("Max AM Resource").__().th()
+              .$class("ui-state-default").__("Used AM Resource").__().th()
+              .$class("ui-state-default").__("Schedulable Apps").__().th()
+              .$class("ui-state-default").__("Non-Schedulable Apps").__().__().__()
               .tbody();
 
       ArrayList<UserInfo> users = lqinfo.getUsers().getUsersList();
@@ -240,11 +240,11 @@ class CapacitySchedulerPage extends RmView {
             .td(resourceUsages.getAMLimit().toString())
             .td(amUsed.toString())
             .td(Integer.toString(userInfo.getNumActiveApplications()))
-            .td(Integer.toString(userInfo.getNumPendingApplications()))._();
+            .td(Integer.toString(userInfo.getNumPendingApplications())).__();
       }
 
-      html.div().$class("usersinfo").h5("Active Users Info")._();
-      tbody._()._();
+      html.div().$class("usersinfo").h5("Active Users Info").__();
+      tbody.__().__();
     }
   }
 
@@ -288,25 +288,25 @@ class CapacitySchedulerPage extends RmView {
             a(_Q).$style(width(absMaxCap * Q_MAX_WIDTH)).
               $title(join("Absolute Capacity:", percent(absCap))).
               span().$style(join(Q_GIVEN, ";font-size:1px;", width(absCap/absMaxCap))).
-                _('.')._().
+            __('.').__().
               span().$style(join(width(absUsedCap/absMaxCap),
                 ";font-size:1px;left:0%;", absUsedCap > absCap ? Q_OVER : Q_UNDER)).
-                _('.')._().
-              span(".q", "Queue: "+info.getQueuePath().substring(5))._().
+            __('.').__().
+              span(".q", "Queue: "+info.getQueuePath().substring(5)).__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._();
+            __(join(percent(used), " used")).__();
 
         csqinfo.qinfo = info;
         if (info.getQueues() == null) {
-          li.ul("#lq").li()._(LeafQueueInfoBlock.class)._()._();
-          li.ul("#lq").li()._(QueueUsersInfoBlock.class)._()._();
+          li.ul("#lq").li().__(LeafQueueInfoBlock.class).__().__();
+          li.ul("#lq").li().__(QueueUsersInfoBlock.class).__().__();
         } else {
-          li._(QueueBlock.class);
+          li.__(QueueBlock.class);
         }
-        li._();
+        li.__();
       }
 
-      ul._();
+      ul.__();
     }
   }
 
@@ -327,7 +327,7 @@ class CapacitySchedulerPage extends RmView {
 
     @Override
     public void render(Block html) {
-      html._(MetricsOverviewTable.class);
+      html.__(MetricsOverviewTable.class);
 
       UserGroupInformation callerUGI = this.getCallerUGI();
       boolean isAdmin = false;
@@ -347,10 +347,10 @@ class CapacitySchedulerPage extends RmView {
           .$style(
               "border-style: solid; border-color: #000000; border-width: 1px;"
                   + " cursor: hand; cursor: pointer; border-radius: 4px")
-          .$onclick("confirmAction()").b("Dump scheduler logs")._().select()
-          .$id("time").option().$value("60")._("1 min")._().option()
-          .$value("300")._("5 min")._().option().$value("600")._("10 min")._()
-          ._()._();
+          .$onclick("confirmAction()").b("Dump scheduler logs").__().select()
+          .$id("time").option().$value("60").__("1 min").__().option()
+          .$value("300").__("5 min").__().option().$value("600").__("10 min").__()
+          .__().__();
 
         StringBuilder script = new StringBuilder();
         script
@@ -377,36 +377,36 @@ class CapacitySchedulerPage extends RmView {
           .append(" console.log(data);").append(" });").append(" }")
           .append("}");
 
-        html.script().$type("text/javascript")._(script.toString())._();
+        html.script().$type("text/javascript").__(script.toString()).__();
       }
 
       UL<DIV<DIV<Hamlet>>> ul = html.
         div("#cs-wrapper.ui-widget").
           div(".ui-widget-header.ui-corner-top").
-            _("Application Queues")._().
+          __("Application Queues").__().
           div("#cs.ui-widget-content.ui-corner-bottom").
             ul();
       if (cs == null) {
         ul.
           li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
-              span().$style(Q_END)._("100% ")._().
-              span(".q", "default")._()._();
+              span().$style(Q_END).__("100% ").__().
+              span(".q", "default").__().__();
       } else {
         ul.
           li().$style("margin-bottom: 1em").
-            span().$style("font-weight: bold")._("Legend:")._().
+            span().$style("font-weight: bold").__("Legend:").__().
             span().$class("qlegend ui-corner-all").$style(Q_GIVEN).
-              _("Capacity")._().
+            __("Capacity").__().
             span().$class("qlegend ui-corner-all").$style(Q_UNDER).
-              _("Used")._().
+            __("Used").__().
             span().$class("qlegend ui-corner-all").$style(Q_OVER).
-              _("Used (over capacity)")._().
+            __("Used (over capacity)").__().
             span().$class("qlegend ui-corner-all ui-state-default").
-              _("Max Capacity")._().
+              __("Max Capacity").__().
             span().$class("qlegend ui-corner-all").$style(ACTIVE_USER).
-            _("Users Requesting Resources")._().
-          _();
+            __("Users Requesting Resources").__().
+          __();
 
         float used = 0;
 
@@ -433,11 +433,11 @@ class CapacitySchedulerPage extends RmView {
           ul.li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
               span().$style(join(width(used), ";left:0%;",
-                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
-              span(".q", "Queue: root")._().
+                  used > 1 ? Q_OVER : Q_UNDER)).__(".").__().
+              span(".q", "Queue: root").__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._().
-            _(QueueBlock.class)._();
+              __(join(percent(used), " used")).__().
+              __(QueueBlock.class).__();
         } else {
           for (RMNodeLabel label : nodeLabelsInfo) {
             csqinfo.qinfo = null;
@@ -453,29 +453,29 @@ class CapacitySchedulerPage extends RmView {
             ul.li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
               span().$style(join(width(used), ";left:0%;",
-                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
-              span(".q", partitionUiTag)._().
+                  used > 1 ? Q_OVER : Q_UNDER)).__(".").__().
+              span(".q", partitionUiTag).__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._()._();
+                __(join(percent(used), " used")).__().__();
 
             //for the queue hierarchy under label
             UL<Hamlet> underLabel = html.ul("#pq");
             underLabel.li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
               span().$style(join(width(used), ";left:0%;",
-                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
-              span(".q", "Queue: root")._().
+                  used > 1 ? Q_OVER : Q_UNDER)).__(".").__().
+              span(".q", "Queue: root").__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._().
-            _(QueueBlock.class)._()._();
+                __(join(percent(used), " used")).__().
+                __(QueueBlock.class).__().__();
           }
         }
       }
-      ul._()._().
+      ul.__().__().
       script().$type("text/javascript").
-          _("$('#cs').hide();")._()._().
-      _(RMAppsBlock.class);
-      html._(HealthBlock.class);
+          __("$('#cs').hide();").__().__().
+          __(RMAppsBlock.class);
+      html.__(HealthBlock.class);
     }
   }
 
@@ -495,13 +495,13 @@ class CapacitySchedulerPage extends RmView {
       div.h4("Aggregate scheduler counts");
       TBODY<TABLE<DIV<Hamlet>>> tbody =
           div.table("#lastrun").thead().$class("ui-widget-header").tr().th()
-            .$class("ui-state-default")._("Total Container Allocations(count)")
-            ._().th().$class("ui-state-default")
-            ._("Total Container Releases(count)")._().th()
+            .$class("ui-state-default").__("Total Container Allocations(count)")
+            .__().th().$class("ui-state-default")
+            .__("Total Container Releases(count)").__().th()
             .$class("ui-state-default")
-            ._("Total Fulfilled Reservations(count)")._().th()
-            .$class("ui-state-default")._("Total Container Preemptions(count)")
-            ._()._()._().tbody();
+            .__("Total Fulfilled Reservations(count)").__().th()
+            .$class("ui-state-default").__("Total Container Preemptions(count)")
+            .__().__().__().tbody();
       tbody
         .$class("ui-widget-content")
         .tr()
@@ -512,15 +512,15 @@ class CapacitySchedulerPage extends RmView {
           String.valueOf(cs.getRootQueueMetrics()
             .getAggegatedReleasedContainers()))
         .td(healthInfo.getAggregateFulFilledReservationsCount().toString())
-        .td(healthInfo.getAggregatePreemptionCount().toString())._()._()._();
+        .td(healthInfo.getAggregatePreemptionCount().toString()).__().__().__();
       div.h4("Last scheduler run");
       tbody =
           div.table("#lastrun").thead().$class("ui-widget-header").tr().th()
-            .$class("ui-state-default")._("Time")._().th()
-            .$class("ui-state-default")._("Allocations(count - resources)")._()
-            .th().$class("ui-state-default")._("Reservations(count - resources)")
-            ._().th().$class("ui-state-default")._("Releases(count - resources)")
-            ._()._()._().tbody();
+            .$class("ui-state-default").__("Time").__().th()
+            .$class("ui-state-default").__("Allocations(count - resources)").__()
+            .th().$class("ui-state-default").__("Reservations(count - resources)")
+            .__().th().$class("ui-state-default").__("Releases(count - resources)")
+            .__().__().__().tbody();
       tbody
         .$class("ui-widget-content")
         .tr()
@@ -533,7 +533,7 @@ class CapacitySchedulerPage extends RmView {
               + healthInfo.getResourcesReserved().toString())
         .td(
           healthInfo.getReleaseCount().toString() + " - "
-              + healthInfo.getResourcesReleased().toString())._()._()._();
+              + healthInfo.getResourcesReleased().toString()).__().__().__();
       Map<String, SchedulerHealth.DetailedInformation> info = new HashMap<>();
       info.put("Allocation", healthInfo.getLastAllocationDetails());
       info.put("Reservation", healthInfo.getLastReservationDetails());
@@ -549,10 +549,10 @@ class CapacitySchedulerPage extends RmView {
         div.h4("Last " + entry.getKey());
         tbody =
             div.table(table).thead().$class("ui-widget-header").tr().th()
-              .$class("ui-state-default")._("Time")._().th()
-              .$class("ui-state-default")._("Container Id")._().th()
-              .$class("ui-state-default")._("Node Id")._().th()
-              .$class("ui-state-default")._("Queue")._()._()._().tbody();
+              .$class("ui-state-default").__("Time").__().th()
+              .$class("ui-state-default").__("Container Id").__().th()
+              .$class("ui-state-default").__("Node Id").__().th()
+              .$class("ui-state-default").__("Queue").__().__().__().tbody();
         SchedulerHealth.DetailedInformation di = entry.getValue();
         if (di.getTimestamp() != 0) {
           containerId = di.getContainerId().toString();
@@ -561,26 +561,26 @@ class CapacitySchedulerPage extends RmView {
         }
         tbody.$class("ui-widget-content").tr()
           .td(Times.format(di.getTimestamp())).td(containerId).td(nodeId)
-          .td(queue)._()._()._();
+          .td(queue).__().__().__();
       }
-      div._();
+      div.__();
     }
   }
 
-  @Override protected void postHead(Page.HTML<_> html) {
+  @Override protected void postHead(Page.HTML<__> html) {
     html.
       style().$type("text/css").
-        _("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
+        __("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
           "#cs ul { list-style: none }",
           "#cs a { font-weight: normal; margin: 2px; position: relative }",
           "#cs a span { font-weight: normal; font-size: 80% }",
           "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }",
           ".qstats { font-weight: normal; font-size: 80%; position: absolute }",
           ".qlegend { font-weight: normal; padding: 0 1em; margin: 1em }",
-          "table.info tr th {width: 50%}")._(). // to center info table
+          "table.info tr th {width: 50%}").__(). // to center info table
       script("/static/jt/jquery.jstree.js").
       script().$type("text/javascript").
-        _("$(function() {",
+        __("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
           "    var callback = { call:reopenQueryNodes }",
@@ -603,8 +603,8 @@ class CapacitySchedulerPage extends RmView {
           "    $('#apps').dataTable().fnFilter(q, 4, true);",
           "  });",
           "  $('#cs').show();",
-          "});")._().
-      _(SchedulerPageUtil.QueueBlockUtil.class);
+          "});").__().
+        __(SchedulerPageUtil.QueueBlockUtil.class);
   }
 
   @Override protected Class<? extends SubView> content() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.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/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
index b8cd1ad..2cd209b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 public class ContainerPage extends RmView {
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String containerId = $(YarnWebParams.CONTAINER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.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/DefaultSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
index d442064..0b0884b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
@@ -26,9 +26,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
 import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.UL;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -53,16 +53,16 @@ class DefaultSchedulerPage extends RmView {
 
     @Override public void render(Block html) {
       info("\'" + sinfo.getQueueName() + "\' Queue Status").
-        _("Queue State:" , sinfo.getState()).
-        _("Minimum Queue Memory Capacity:" , Long.toString(sinfo.getMinQueueMemoryCapacity())).
-        _("Maximum Queue Memory Capacity:" , Long.toString(sinfo.getMaxQueueMemoryCapacity())).
-        _("Number of Nodes:" , Integer.toString(sinfo.getNumNodes())).
-        _("Used Node Capacity:" , Integer.toString(sinfo.getUsedNodeCapacity())).
-        _("Available Node Capacity:" , Integer.toString(sinfo.getAvailNodeCapacity())).
-        _("Total Node Capacity:" , Integer.toString(sinfo.getTotalNodeCapacity())).
-        _("Number of Node Containers:" , Integer.toString(sinfo.getNumContainers()));
-
-      html._(InfoBlock.class);
+          __("Queue State:" , sinfo.getState()).
+          __("Minimum Queue Memory Capacity:" , Long.toString(sinfo.getMinQueueMemoryCapacity())).
+          __("Maximum Queue Memory Capacity:" , Long.toString(sinfo.getMaxQueueMemoryCapacity())).
+          __("Number of Nodes:" , Integer.toString(sinfo.getNumNodes())).
+          __("Used Node Capacity:" , Integer.toString(sinfo.getUsedNodeCapacity())).
+          __("Available Node Capacity:" , Integer.toString(sinfo.getAvailNodeCapacity())).
+          __("Total Node Capacity:" , Integer.toString(sinfo.getTotalNodeCapacity())).
+          __("Number of Node Containers:" , Integer.toString(sinfo.getNumContainers()));
+
+      html.__(InfoBlock.class);
     }
   }
 
@@ -77,11 +77,11 @@ class DefaultSchedulerPage extends RmView {
 
     @Override
     public void render(Block html) {
-      html._(MetricsOverviewTable.class);
+      html.__(MetricsOverviewTable.class);
       UL<DIV<DIV<Hamlet>>> ul = html.
         div("#cs-wrapper.ui-widget").
           div(".ui-widget-header.ui-corner-top").
-            _("FifoScheduler Queue")._().
+          __("FifoScheduler Queue").__().
           div("#cs.ui-widget-content.ui-corner-bottom").
             ul();
 
@@ -89,8 +89,8 @@ class DefaultSchedulerPage extends RmView {
         ul.
           li().
             a(_Q).$style(width(WIDTH_F)).
-              span().$style(Q_END)._("100% ")._().
-              span(".q", "default")._()._();
+              span().$style(Q_END).__("100% ").__().
+              span(".q", "default").__().__();
       } else {
         float used = sinfo.getUsedCapacity();
         float set = sinfo.getCapacity();
@@ -99,33 +99,33 @@ class DefaultSchedulerPage extends RmView {
           li().
             a(_Q).$style(width(WIDTH_F)).
               $title(join("used:", percent(used))).
-              span().$style(Q_END)._("100%")._().
+              span().$style(Q_END).__("100%").__().
               span().$style(join(width(delta), ';', used > set ? OVER : UNDER,
-                ';', used > set ? left(set) : left(used)))._(".")._().
-              span(".q", sinfo.getQueueName())._().
-            _(QueueInfoBlock.class)._();
+                ';', used > set ? left(set) : left(used))).__(".").__().
+              span(".q", sinfo.getQueueName()).__().
+            __(QueueInfoBlock.class).__();
       }
 
-      ul._()._().
+      ul.__().__().
       script().$type("text/javascript").
-          _("$('#cs').hide();")._()._().
-      _(AppsBlock.class);
+          __("$('#cs').hide();").__().__().
+          __(AppsBlock.class);
     }
   }
 
 
-  @Override protected void postHead(Page.HTML<_> html) {
+  @Override protected void postHead(Page.HTML<__> html) {
     html.
       style().$type("text/css").
-        _("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
+        __("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
           "#cs ul { list-style: none }",
           "#cs a { font-weight: normal; margin: 2px; position: relative }",
           "#cs a span { font-weight: normal; font-size: 80% }",
           "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }",
-          "table.info tr th {width: 50%}")._(). // to center info table
+          "table.info tr th {width: 50%}").__(). // to center info table
       script("/static/jt/jquery.jstree.js").
       script().$type("text/javascript").
-        _("$(function() {",
+        __("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
           "    data.inst.open_all(); }).",
@@ -142,7 +142,7 @@ class DefaultSchedulerPage extends RmView {
           "    $('#apps').dataTable().fnFilter(q, 4);",
           "  });",
           "  $('#cs').show();",
-          "});")._();
+          "});").__();
   }
 
   @Override protected Class<? extends SubView> content() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ErrorBlock.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/ErrorBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ErrorBlock.java
index 963e53f..6fe5c3a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ErrorBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ErrorBlock.java
@@ -34,6 +34,6 @@ public class ErrorBlock extends HtmlBlock {
 
   @Override
   protected void render(Block html) {
-    html.p()._($(ERROR_MESSAGE))._();
+    html.p().__($(ERROR_MESSAGE)).__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.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/FairSchedulerAppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
index b7a7a93..ac88f86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
@@ -41,9 +41,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
@@ -98,7 +98,7 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
             th(".reservedCpu", "Reserved CPU VCores").
             th(".reservedMemory", "Reserved Memory MB").
             th(".progress", "Progress").
-            th(".ui", "Tracking UI")._()._().
+            th(".ui", "Tracking UI").__().__().
         tbody();
     Collection<YarnApplicationState> reqAppStates = null;
     String reqStateString = $(APP_STATE);
@@ -168,8 +168,8 @@ public class FairSchedulerAppsBlock extends HtmlBlock {
     }
     appsTableData.append("]");
     html.script().$type("text/javascript").
-    _("var appsTableData=" + appsTableData)._();
+        __("var appsTableData=" + appsTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.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/FairSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
index 5f46841..ffa4594 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
@@ -31,10 +31,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerQue
 import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.LI;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.UL;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -70,21 +70,21 @@ public class FairSchedulerPage extends RmView {
     @Override
     protected void render(Block html) {
       ResponseInfo ri = info("\'" + qinfo.getQueueName() + "\' Queue Status").
-          _("Used Resources:", qinfo.getUsedResources().toString()).
-          _("Demand Resources:", qinfo.getDemandResources().toString()).
-          _("Num Active Applications:", qinfo.getNumActiveApplications()).
-          _("Num Pending Applications:", qinfo.getNumPendingApplications()).
-          _("Min Resources:", qinfo.getMinResources().toString()).
-          _("Max Resources:", qinfo.getMaxResources().toString()).
-          _("Reserved Resources:", qinfo.getReservedResources().toString());
+          __("Used Resources:", qinfo.getUsedResources().toString()).
+          __("Demand Resources:", qinfo.getDemandResources().toString()).
+          __("Num Active Applications:", qinfo.getNumActiveApplications()).
+          __("Num Pending Applications:", qinfo.getNumPendingApplications()).
+          __("Min Resources:", qinfo.getMinResources().toString()).
+          __("Max Resources:", qinfo.getMaxResources().toString()).
+          __("Reserved Resources:", qinfo.getReservedResources().toString());
       int maxApps = qinfo.getMaxApplications();
       if (maxApps < Integer.MAX_VALUE) {
-          ri._("Max Running Applications:", qinfo.getMaxApplications());
+        ri.__("Max Running Applications:", qinfo.getMaxApplications());
       }
-      ri._(STEADY_FAIR_SHARE + ":", qinfo.getSteadyFairShare().toString());
-      ri._(INSTANTANEOUS_FAIR_SHARE + ":", qinfo.getFairShare().toString());
-      ri._("Preemptable:", qinfo.isPreemptable());
-      html._(InfoBlock.class);
+      ri.__(STEADY_FAIR_SHARE + ":", qinfo.getSteadyFairShare().toString());
+      ri.__(INSTANTANEOUS_FAIR_SHARE + ":", qinfo.getFairShare().toString());
+      ri.__("Preemptable:", qinfo.isPreemptable());
+      html.__(InfoBlock.class);
 
       // clear the info contents so this queue's info doesn't accumulate into another queue's info
       ri.clear();
@@ -102,17 +102,17 @@ public class FairSchedulerPage extends RmView {
     @Override
     protected void render(Block html) {
       ResponseInfo ri = info("\'" + qinfo.getQueueName() + "\' Queue Status").
-          _("Used Resources:", qinfo.getUsedResources().toString()).
-          _("Min Resources:", qinfo.getMinResources().toString()).
-          _("Max Resources:", qinfo.getMaxResources().toString()).
-          _("Reserved Resources:", qinfo.getReservedResources().toString());
+          __("Used Resources:", qinfo.getUsedResources().toString()).
+          __("Min Resources:", qinfo.getMinResources().toString()).
+          __("Max Resources:", qinfo.getMaxResources().toString()).
+          __("Reserved Resources:", qinfo.getReservedResources().toString());
       int maxApps = qinfo.getMaxApplications();
       if (maxApps < Integer.MAX_VALUE) {
-          ri._("Max Running Applications:", qinfo.getMaxApplications());
+        ri.__("Max Running Applications:", qinfo.getMaxApplications());
       }
-      ri._(STEADY_FAIR_SHARE + ":", qinfo.getSteadyFairShare().toString());
-      ri._(INSTANTANEOUS_FAIR_SHARE + ":", qinfo.getFairShare().toString());
-      html._(InfoBlock.class);
+      ri.__(STEADY_FAIR_SHARE + ":", qinfo.getSteadyFairShare().toString());
+      ri.__(INSTANTANEOUS_FAIR_SHARE + ":", qinfo.getFairShare().toString());
+      html.__(InfoBlock.class);
 
       // clear the info contents so this queue's info doesn't accumulate into another queue's info
       ri.clear();
@@ -141,28 +141,28 @@ public class FairSchedulerPage extends RmView {
               $title(join(join(STEADY_FAIR_SHARE + ":", percent(steadyFairShare)),
                   join(" " + INSTANTANEOUS_FAIR_SHARE + ":", percent(instantaneousFairShare)))).
               span().$style(join(Q_GIVEN, ";font-size:1px;", width(steadyFairShare / capacity))).
-                _('.')._().
+            __('.').__().
               span().$style(join(Q_INSTANTANEOUS_FS, ";font-size:1px;",
                   width(instantaneousFairShare/capacity))).
-                _('.')._().
+            __('.').__().
               span().$style(join(width(used/capacity),
                 ";font-size:1px;left:0%;", used > instantaneousFairShare ? Q_OVER : Q_UNDER)).
-                _('.')._().
-              span(".q", info.getQueueName())._().
+            __('.').__().
+              span(".q", info.getQueueName()).__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._();
+            __(join(percent(used), " used")).__();
 
         fsqinfo.qinfo = info;
         if (info instanceof FairSchedulerLeafQueueInfo) {
-          li.ul("#lq").li()._(LeafQueueBlock.class)._()._();
+          li.ul("#lq").li().__(LeafQueueBlock.class).__().__();
         } else {
-          li.ul("#lq").li()._(ParentQueueBlock.class)._()._();
-          li._(QueueBlock.class);
+          li.ul("#lq").li().__(ParentQueueBlock.class).__().__();
+          li.__(QueueBlock.class);
         }
-        li._();
+        li.__();
       }
 
-      ul._();
+      ul.__();
     }
   }
   
@@ -177,19 +177,19 @@ public class FairSchedulerPage extends RmView {
 
     @Override
     public void render(Block html) {
-      html._(MetricsOverviewTable.class);
+      html.__(MetricsOverviewTable.class);
       UL<DIV<DIV<Hamlet>>> ul = html.
         div("#cs-wrapper.ui-widget").
           div(".ui-widget-header.ui-corner-top").
-            _("Application Queues")._().
+          __("Application Queues").__().
           div("#cs.ui-widget-content.ui-corner-bottom").
             ul();
       if (fs == null) {
         ul.
           li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
-              span().$style(Q_END)._("100% ")._().
-              span(".q", "default")._()._();
+              span().$style(Q_END).__("100% ").__().
+              span(".q", "default").__().__();
       } else {
         FairSchedulerInfo sinfo = new FairSchedulerInfo(fs);
         fsqinfo.qinfo = sinfo.getRootQueueInfo();
@@ -197,52 +197,52 @@ public class FairSchedulerPage extends RmView {
 
         ul.
           li().$style("margin-bottom: 1em").
-            span().$style("font-weight: bold")._("Legend:")._().
+            span().$style("font-weight: bold").__("Legend:").__().
             span().$class("qlegend ui-corner-all").$style(Q_GIVEN).
               $title("The steady fair shares consider all queues, " +
                   "both active (with running applications) and inactive.").
-              _(STEADY_FAIR_SHARE)._().
+            __(STEADY_FAIR_SHARE).__().
             span().$class("qlegend ui-corner-all").$style(Q_INSTANTANEOUS_FS).
               $title("The instantaneous fair shares consider only active " +
                   "queues (with running applications).").
-              _(INSTANTANEOUS_FAIR_SHARE)._().
+            __(INSTANTANEOUS_FAIR_SHARE).__().
             span().$class("qlegend ui-corner-all").$style(Q_UNDER).
-              _("Used")._().
+            __("Used").__().
             span().$class("qlegend ui-corner-all").$style(Q_OVER).
-              _("Used (over fair share)")._().
+            __("Used (over fair share)").__().
             span().$class("qlegend ui-corner-all ui-state-default").
-              _("Max Capacity")._().
-        _().
+            __("Max Capacity").__().
+            __().
           li().
             a(_Q).$style(width(Q_MAX_WIDTH)).
               span().$style(join(width(used), ";left:0%;",
-                  used > 1 ? Q_OVER : Q_UNDER))._(".")._().
-              span(".q", "root")._().
+                  used > 1 ? Q_OVER : Q_UNDER)).__(".").__().
+              span(".q", "root").__().
             span().$class("qstats").$style(left(Q_STATS_POS)).
-              _(join(percent(used), " used"))._().
-            _(QueueBlock.class)._();
+            __(join(percent(used), " used")).__().
+            __(QueueBlock.class).__();
       }
-      ul._()._().
+      ul.__().__().
       script().$type("text/javascript").
-          _("$('#cs').hide();")._()._().
-      _(FairSchedulerAppsBlock.class);
+          __("$('#cs').hide();").__().__().
+          __(FairSchedulerAppsBlock.class);
     }
   }
   
-  @Override protected void postHead(Page.HTML<_> html) {
+  @Override protected void postHead(Page.HTML<__> html) {
     html.
       style().$type("text/css").
-        _("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
+        __("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
           "#cs ul { list-style: none }",
           "#cs a { font-weight: normal; margin: 2px; position: relative }",
           "#cs a span { font-weight: normal; font-size: 80% }",
           "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }",
           ".qstats { font-weight: normal; font-size: 80%; position: absolute }",
           ".qlegend { font-weight: normal; padding: 0 1em; margin: 1em }",
-          "table.info tr th {width: 50%}")._(). // to center info table
+          "table.info tr th {width: 50%}").__(). // to center info table
       script("/static/jt/jquery.jstree.js").
       script().$type("text/javascript").
-        _("$(function() {",
+        __("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
           "    var callback = { call:reopenQueryNodes }",
@@ -262,8 +262,8 @@ public class FairSchedulerPage extends RmView {
           "    $('#apps').dataTable().fnFilter(q, 4, true);",
           "  });",
           "  $('#cs').show();",
-          "});")._().
-        _(SchedulerPageUtil.QueueBlockUtil.class);
+          "});").__().
+        __(SchedulerPageUtil.QueueBlockUtil.class);
   }
   
   @Override protected Class<? extends SubView> content() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.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/MetricsOverviewTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index fe7b247..f6b1a94 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsIn
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.UserMetricsInfo;
 
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -61,19 +61,19 @@ public class MetricsOverviewTable extends HtmlBlock {
     table("#metricsoverview").
     thead().$class("ui-widget-header").
       tr().
-        th().$class("ui-state-default")._("Apps Submitted")._().
-        th().$class("ui-state-default")._("Apps Pending")._().
-        th().$class("ui-state-default")._("Apps Running")._().
-        th().$class("ui-state-default")._("Apps Completed")._().
-        th().$class("ui-state-default")._("Containers Running")._().
-        th().$class("ui-state-default")._("Memory Used")._().
-        th().$class("ui-state-default")._("Memory Total")._().
-        th().$class("ui-state-default")._("Memory Reserved")._().
-        th().$class("ui-state-default")._("VCores Used")._().
-        th().$class("ui-state-default")._("VCores Total")._().
-        th().$class("ui-state-default")._("VCores Reserved")._().
-      _().
-    _().
+        th().$class("ui-state-default").__("Apps Submitted").__().
+        th().$class("ui-state-default").__("Apps Pending").__().
+        th().$class("ui-state-default").__("Apps Running").__().
+        th().$class("ui-state-default").__("Apps Completed").__().
+        th().$class("ui-state-default").__("Containers Running").__().
+        th().$class("ui-state-default").__("Memory Used").__().
+        th().$class("ui-state-default").__("Memory Total").__().
+        th().$class("ui-state-default").__("Memory Reserved").__().
+        th().$class("ui-state-default").__("VCores Used").__().
+        th().$class("ui-state-default").__("VCores Total").__().
+        th().$class("ui-state-default").__("VCores Reserved").__().
+        __().
+        __().
     tbody().$class("ui-widget-content").
       tr().
         td(String.valueOf(clusterMetrics.getAppsSubmitted())).
@@ -92,33 +92,33 @@ public class MetricsOverviewTable extends HtmlBlock {
         td(String.valueOf(clusterMetrics.getAllocatedVirtualCores())).
         td(String.valueOf(clusterMetrics.getTotalVirtualCores())).
         td(String.valueOf(clusterMetrics.getReservedVirtualCores())).
-      _().
-    _()._();
+        __().
+        __().__();
 
     div.h3("Cluster Nodes Metrics").
     table("#nodemetricsoverview").
     thead().$class("ui-widget-header").
       tr().
-        th().$class("ui-state-default")._("Active Nodes")._().
-        th().$class("ui-state-default")._("Decommissioning Nodes")._().
-        th().$class("ui-state-default")._("Decommissioned Nodes")._().
-        th().$class("ui-state-default")._("Lost Nodes")._().
-        th().$class("ui-state-default")._("Unhealthy Nodes")._().
-        th().$class("ui-state-default")._("Rebooted Nodes")._().
-        th().$class("ui-state-default")._("Shutdown Nodes")._().
-      _().
-    _().
+        th().$class("ui-state-default").__("Active Nodes").__().
+        th().$class("ui-state-default").__("Decommissioning Nodes").__().
+        th().$class("ui-state-default").__("Decommissioned Nodes").__().
+        th().$class("ui-state-default").__("Lost Nodes").__().
+        th().$class("ui-state-default").__("Unhealthy Nodes").__().
+        th().$class("ui-state-default").__("Rebooted Nodes").__().
+        th().$class("ui-state-default").__("Shutdown Nodes").__().
+        __().
+        __().
     tbody().$class("ui-widget-content").
       tr().
-        td().a(url("nodes"),String.valueOf(clusterMetrics.getActiveNodes()))._().
-        td().a(url("nodes/decommissioning"), String.valueOf(clusterMetrics.getDecommissioningNodes()))._().
-        td().a(url("nodes/decommissioned"),String.valueOf(clusterMetrics.getDecommissionedNodes()))._().
-        td().a(url("nodes/lost"),String.valueOf(clusterMetrics.getLostNodes()))._().
-        td().a(url("nodes/unhealthy"),String.valueOf(clusterMetrics.getUnhealthyNodes()))._().
-        td().a(url("nodes/rebooted"),String.valueOf(clusterMetrics.getRebootedNodes()))._().
-        td().a(url("nodes/shutdown"),String.valueOf(clusterMetrics.getShutdownNodes()))._().
-      _().
-    _()._();
+        td().a(url("nodes"), String.valueOf(clusterMetrics.getActiveNodes())).__().
+        td().a(url("nodes/decommissioning"), String.valueOf(clusterMetrics.getDecommissioningNodes())).__().
+        td().a(url("nodes/decommissioned"), String.valueOf(clusterMetrics.getDecommissionedNodes())).__().
+        td().a(url("nodes/lost"), String.valueOf(clusterMetrics.getLostNodes())).__().
+        td().a(url("nodes/unhealthy"), String.valueOf(clusterMetrics.getUnhealthyNodes())).__().
+        td().a(url("nodes/rebooted"), String.valueOf(clusterMetrics.getRebootedNodes())).__().
+        td().a(url("nodes/shutdown"), String.valueOf(clusterMetrics.getShutdownNodes())).__().
+        __().
+        __().__();
 
     String user = request().getRemoteUser();
     if (user != null) {
@@ -128,21 +128,21 @@ public class MetricsOverviewTable extends HtmlBlock {
         table("#usermetricsoverview").
         thead().$class("ui-widget-header").
           tr().
-            th().$class("ui-state-default")._("Apps Submitted")._().
-            th().$class("ui-state-default")._("Apps Pending")._().
-            th().$class("ui-state-default")._("Apps Running")._().
-            th().$class("ui-state-default")._("Apps Completed")._().
-            th().$class("ui-state-default")._("Containers Running")._().
-            th().$class("ui-state-default")._("Containers Pending")._().
-            th().$class("ui-state-default")._("Containers Reserved")._().
-            th().$class("ui-state-default")._("Memory Used")._().
-            th().$class("ui-state-default")._("Memory Pending")._().
-            th().$class("ui-state-default")._("Memory Reserved")._().
-            th().$class("ui-state-default")._("VCores Used")._().
-            th().$class("ui-state-default")._("VCores Pending")._().
-            th().$class("ui-state-default")._("VCores Reserved")._().
-          _().
-        _().
+            th().$class("ui-state-default").__("Apps Submitted").__().
+            th().$class("ui-state-default").__("Apps Pending").__().
+            th().$class("ui-state-default").__("Apps Running").__().
+            th().$class("ui-state-default").__("Apps Completed").__().
+            th().$class("ui-state-default").__("Containers Running").__().
+            th().$class("ui-state-default").__("Containers Pending").__().
+            th().$class("ui-state-default").__("Containers Reserved").__().
+            th().$class("ui-state-default").__("Memory Used").__().
+            th().$class("ui-state-default").__("Memory Pending").__().
+            th().$class("ui-state-default").__("Memory Reserved").__().
+            th().$class("ui-state-default").__("VCores Used").__().
+            th().$class("ui-state-default").__("VCores Pending").__().
+            th().$class("ui-state-default").__("VCores Reserved").__().
+            __().
+            __().
         tbody().$class("ui-widget-content").
           tr().
             td(String.valueOf(userMetrics.getAppsSubmitted())).
@@ -163,8 +163,8 @@ public class MetricsOverviewTable extends HtmlBlock {
             td(String.valueOf(userMetrics.getAllocatedVirtualCores())).
             td(String.valueOf(userMetrics.getPendingVirtualCores())).
             td(String.valueOf(userMetrics.getReservedVirtualCores())).
-          _().
-        _()._();
+            __().
+            __().__();
         
       }
     }
@@ -175,14 +175,14 @@ public class MetricsOverviewTable extends HtmlBlock {
     table("#schedulermetricsoverview").
     thead().$class("ui-widget-header").
       tr().
-        th().$class("ui-state-default")._("Scheduler Type")._().
-        th().$class("ui-state-default")._("Scheduling Resource Type")._().
-        th().$class("ui-state-default")._("Minimum Allocation")._().
-        th().$class("ui-state-default")._("Maximum Allocation")._().
+        th().$class("ui-state-default").__("Scheduler Type").__().
+        th().$class("ui-state-default").__("Scheduling Resource Type").__().
+        th().$class("ui-state-default").__("Minimum Allocation").__().
+        th().$class("ui-state-default").__("Maximum Allocation").__().
         th().$class("ui-state-default")
-            ._("Maximum Cluster Application Priority")._().
-      _().
-    _().
+            .__("Maximum Cluster Application Priority").__().
+        __().
+        __().
     tbody().$class("ui-widget-content").
       tr().
         td(String.valueOf(schedulerInfo.getSchedulerType())).
@@ -190,9 +190,9 @@ public class MetricsOverviewTable extends HtmlBlock {
         td(schedulerInfo.getMinAllocation().toString()).
         td(schedulerInfo.getMaxAllocation().toString()).
         td(String.valueOf(schedulerInfo.getMaxClusterLevelAppPriority())).
-      _().
-    _()._();
+        __().
+        __().__();
 
-    div._();
+    div.__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.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/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
index ca55175..1993f6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java
@@ -23,10 +23,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.LI;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.UL;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 public class NavBlock extends HtmlBlock {
@@ -45,29 +45,29 @@ public class NavBlock extends HtmlBlock {
       div("#nav").
         h3("Cluster").
         ul().
-          li().a(url("cluster"), "About")._().
-          li().a(url("nodes"), "Nodes")._().
-          li().a(url("nodelabels"), "Node Labels")._();
+          li().a(url("cluster"), "About").__().
+          li().a(url("nodes"), "Nodes").__().
+          li().a(url("nodelabels"), "Node Labels").__();
     UL<LI<UL<DIV<Hamlet>>>> subAppsList = mainList.
           li().a(url("apps"), "Applications").
             ul();
-    subAppsList.li()._();
+    subAppsList.li().__();
     for (YarnApplicationState state : YarnApplicationState.values()) {
       subAppsList.
-              li().a(url("apps", state.toString()), state.toString())._();
+              li().a(url("apps", state.toString()), state.toString()).__();
     }
-    subAppsList._()._();
+    subAppsList.__().__();
     UL<DIV<Hamlet>> tools = mainList.
-          li().a(url("scheduler"), "Scheduler")._()._().
+          li().a(url("scheduler"), "Scheduler").__().__().
         h3("Tools").ul();
-    tools.li().a("/conf", "Configuration")._().
-          li().a("/logs", "Local logs")._().
-          li().a("/stacks", "Server stacks")._().
-          li().a("/jmx?qry=Hadoop:*", "Server metrics")._();
+    tools.li().a("/conf", "Configuration").__().
+          li().a("/logs", "Local logs").__().
+          li().a("/stacks", "Server stacks").__().
+          li().a("/jmx?qry=Hadoop:*", "Server metrics").__();
 
     if (addErrorsAndWarningsLink) {
-      tools.li().a(url("errors-and-warnings"), "Errors/Warnings")._();
+      tools.li().a(url("errors-and-warnings"), "Errors/Warnings").__();
     }
-    tools._()._();
+    tools.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.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/NodeLabelsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
index ea85d13..6ff7628 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -53,7 +53,7 @@ public class NodeLabelsPage extends RmView {
           th(".type", "Label Type").
           th(".numOfActiveNMs", "Num Of Active NMs").
           th(".totalResource", "Total Resource").
-          _()._().
+          __().__().
           tbody();
   
       RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
@@ -71,17 +71,17 @@ public class NodeLabelsPage extends RmView {
           .a(url("nodes",
               "?" + YarnWebParams.NODE_LABEL + "=" + info.getLabelName()),
               String.valueOf(nActiveNMs))
-           ._();
+           .__();
         } else {
           row = row.td(String.valueOf(nActiveNMs));
         }
-        row.td(info.getResource().toString())._();
+        row.td(info.getResource().toString()).__();
       }
-      tbody._()._();
+      tbody.__().__();
     }
   }
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String title = "Node labels of the cluster";
     setTitle(title);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.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/NodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
index c03df63..d0e384d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodesPage.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import java.util.Collection;
@@ -63,7 +63,7 @@ class NodesPage extends RmView {
 
     @Override
     protected void render(Block html) {
-      html._(MetricsOverviewTable.class);
+      html.__(MetricsOverviewTable.class);
 
       ResourceScheduler sched = rm.getResourceScheduler();
 
@@ -98,7 +98,7 @@ class NodesPage extends RmView {
       }
 
       TBODY<TABLE<Hamlet>> tbody =
-          trbody.th(".nodeManagerVersion", "Version")._()._().tbody();
+          trbody.th(".nodeManagerVersion", "Version").__().__().tbody();
 
       NodeState stateFilter = null;
       if (type != null && !type.isEmpty()) {
@@ -201,13 +201,13 @@ class NodesPage extends RmView {
       }
       nodeTableData.append("]");
       html.script().$type("text/javascript")
-          ._("var nodeTableData=" + nodeTableData)._();
-      tbody._()._();
+          .__("var nodeTableData=" + nodeTableData).__();
+      tbody.__().__();
     }
   }
 
   @Override
-  protected void preHead(Page.HTML<_> html) {
+  protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String type = $(NODE_STATE);
     String title = "Nodes of the cluster";


---------------------------------------------------------------------
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: HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.

Posted by in...@apache.org.
HADOOP-11875. [JDK9] Adding a second copy of Hamlet without _ as a one-character identifier.


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

Branch: refs/heads/HDFS-10467
Commit: 38c6fa5c7a61c7f6d4d2db5f12f9c60d477fb397
Parents: c6330f2
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jul 28 08:57:34 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jul 28 09:10:52 2017 +0900

----------------------------------------------------------------------
 .../mapreduce/v2/app/webapp/AppController.java  |    10 +-
 .../hadoop/mapreduce/v2/app/webapp/AppView.java |     4 +-
 .../mapreduce/v2/app/webapp/ConfBlock.java      |    36 +-
 .../mapreduce/v2/app/webapp/CountersBlock.java  |    38 +-
 .../mapreduce/v2/app/webapp/CountersPage.java   |     4 +-
 .../mapreduce/v2/app/webapp/InfoPage.java       |     2 +-
 .../mapreduce/v2/app/webapp/JobBlock.java       |    95 +-
 .../mapreduce/v2/app/webapp/JobConfPage.java    |     3 +-
 .../hadoop/mapreduce/v2/app/webapp/JobPage.java |     2 +-
 .../mapreduce/v2/app/webapp/JobsBlock.java      |    24 +-
 .../mapreduce/v2/app/webapp/NavBlock.java       |    38 +-
 .../v2/app/webapp/SingleCounterBlock.java       |    28 +-
 .../v2/app/webapp/SingleCounterPage.java        |     3 +-
 .../mapreduce/v2/app/webapp/TaskPage.java       |    20 +-
 .../mapreduce/v2/app/webapp/TasksBlock.java     |    12 +-
 .../mapreduce/v2/app/webapp/TasksPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsAboutPage.java     |     7 +-
 .../mapreduce/v2/hs/webapp/HsConfPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsCountersPage.java  |     4 +-
 .../mapreduce/v2/hs/webapp/HsJobBlock.java      |    98 +-
 .../mapreduce/v2/hs/webapp/HsJobPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsJobsBlock.java     |    46 +-
 .../mapreduce/v2/hs/webapp/HsLogsPage.java      |     2 +-
 .../mapreduce/v2/hs/webapp/HsNavBlock.java      |    30 +-
 .../v2/hs/webapp/HsSingleCounterPage.java       |     2 +-
 .../mapreduce/v2/hs/webapp/HsTaskPage.java      |    53 +-
 .../mapreduce/v2/hs/webapp/HsTasksBlock.java    |    57 +-
 .../mapreduce/v2/hs/webapp/HsTasksPage.java     |     2 +-
 .../hadoop/mapreduce/v2/hs/webapp/HsView.java   |     4 +-
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    22 +
 .../apache/hadoop/yarn/webapp/ResponseInfo.java |     4 +-
 .../hadoop/yarn/webapp/example/HelloWorld.java  |     4 +-
 .../hadoop/yarn/webapp/example/MyApp.java       |     6 +-
 .../hadoop/yarn/webapp/hamlet/Hamlet.java       |     4 +
 .../hadoop/yarn/webapp/hamlet/HamletGen.java    |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletImpl.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/HamletSpec.java   |     2 +
 .../hadoop/yarn/webapp/hamlet/package-info.java |     6 +
 .../hadoop/yarn/webapp/hamlet2/Hamlet.java      | 30557 +++++++++++++++++
 .../hadoop/yarn/webapp/hamlet2/HamletGen.java   |   449 +
 .../hadoop/yarn/webapp/hamlet2/HamletImpl.java  |   385 +
 .../hadoop/yarn/webapp/hamlet2/HamletSpec.java  |  3101 ++
 .../yarn/webapp/hamlet2/package-info.java       |    27 +
 .../yarn/webapp/log/AggregatedLogsBlock.java    |    66 +-
 .../yarn/webapp/log/AggregatedLogsNavBlock.java |     4 +-
 .../yarn/webapp/log/AggregatedLogsPage.java     |     2 +-
 .../hadoop/yarn/webapp/view/ErrorPage.java      |    12 +-
 .../hadoop/yarn/webapp/view/FooterBlock.java    |     2 +-
 .../hadoop/yarn/webapp/view/HeaderBlock.java    |     6 +-
 .../hadoop/yarn/webapp/view/HtmlBlock.java      |     2 +-
 .../hadoop/yarn/webapp/view/HtmlPage.java       |    12 +-
 .../hadoop/yarn/webapp/view/InfoBlock.java      |    28 +-
 .../hadoop/yarn/webapp/view/JQueryUI.java       |    14 +-
 .../hadoop/yarn/webapp/view/LipsumBlock.java    |     4 +-
 .../hadoop/yarn/webapp/view/NavBlock.java       |    10 +-
 .../yarn/webapp/view/TwoColumnCssLayout.java    |    20 +-
 .../yarn/webapp/view/TwoColumnLayout.java       |    20 +-
 .../apache/hadoop/yarn/webapp/TestSubViews.java |    10 +-
 .../apache/hadoop/yarn/webapp/TestWebApp.java   |    28 +-
 .../hadoop/yarn/webapp/view/TestHtmlBlock.java  |    10 +-
 .../hadoop/yarn/webapp/view/TestHtmlPage.java   |     9 +-
 .../hadoop/yarn/webapp/view/TestInfoBlock.java  |     6 +-
 .../yarn/webapp/view/TestTwoColumnCssPage.java  |     6 +-
 .../webapp/AHSErrorsAndWarningsPage.java        |     2 +-
 .../webapp/AHSLogsPage.java                     |     2 +-
 .../webapp/AHSView.java                         |     4 +-
 .../webapp/AboutBlock.java                      |     8 +-
 .../webapp/AboutPage.java                       |     5 +-
 .../webapp/AppAttemptPage.java                  |     2 +-
 .../webapp/AppPage.java                         |     2 +-
 .../webapp/ContainerPage.java                   |     2 +-
 .../webapp/NavBlock.java                        |    28 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |    31 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |    66 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |    14 +-
 .../yarn/server/webapp/ContainerBlock.java      |    23 +-
 .../server/webapp/ErrorsAndWarningsBlock.java   |    50 +-
 .../nodemanager/webapp/AllApplicationsPage.java |    28 +-
 .../nodemanager/webapp/AllContainersPage.java   |    30 +-
 .../nodemanager/webapp/ApplicationPage.java     |    25 +-
 .../nodemanager/webapp/ContainerLogsPage.java   |    17 +-
 .../nodemanager/webapp/ContainerPage.java       |    31 +-
 .../webapp/NMErrorsAndWarningsPage.java         |     2 +-
 .../yarn/server/nodemanager/webapp/NMView.java  |     4 +-
 .../server/nodemanager/webapp/NavBlock.java     |    28 +-
 .../server/nodemanager/webapp/NodePage.java     |    28 +-
 .../resourcemanager/webapp/AboutBlock.java      |    20 +-
 .../resourcemanager/webapp/AboutPage.java       |     2 +-
 .../resourcemanager/webapp/AppAttemptPage.java  |     2 +-
 .../webapp/AppLogAggregationStatusPage.java     |     2 +-
 .../server/resourcemanager/webapp/AppPage.java  |     2 +-
 .../webapp/AppsBlockWithMetrics.java            |     5 +-
 .../webapp/CapacitySchedulerPage.java           |   220 +-
 .../resourcemanager/webapp/ContainerPage.java   |     2 +-
 .../webapp/DefaultSchedulerPage.java            |    58 +-
 .../resourcemanager/webapp/ErrorBlock.java      |     2 +-
 .../webapp/FairSchedulerAppsBlock.java          |    12 +-
 .../webapp/FairSchedulerPage.java               |   116 +-
 .../webapp/MetricsOverviewTable.java            |   124 +-
 .../server/resourcemanager/webapp/NavBlock.java |    34 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |    18 +-
 .../resourcemanager/webapp/NodesPage.java       |    16 +-
 .../webapp/RMAppAttemptBlock.java               |    48 +-
 .../resourcemanager/webapp/RMAppBlock.java      |    28 +-
 .../webapp/RMAppLogAggregationStatusBlock.java  |    34 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |    15 +-
 .../webapp/RMErrorsAndWarningsPage.java         |     2 +-
 .../webapp/RedirectionErrorPage.java            |     2 +-
 .../server/resourcemanager/webapp/RmView.java   |     4 +-
 .../webapp/SchedulerPageUtil.java               |     8 +-
 .../webapp/SCMOverviewPage.java                 |    26 +-
 .../hadoop/yarn/server/webproxy/ProxyUtils.java |    16 +-
 .../server/webproxy/WebAppProxyServlet.java     |    14 +-
 113 files changed, 35635 insertions(+), 1101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
index 1c90cb9..6db1274 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
@@ -84,11 +84,11 @@ public class AppController extends Controller implements AMParams {
   public void info() {
     AppInfo info = new AppInfo(app, app.context);
     info("Application Master Overview").
-      _("Application ID:", info.getId()).
-      _("Application Name:", info.getName()).
-      _("User:", info.getUser()).
-      _("Started on:", Times.format(info.getStartTime())).
-      _("Elasped: ", org.apache.hadoop.util.StringUtils.formatTime(
+        __("Application ID:", info.getId()).
+        __("Application Name:", info.getName()).
+        __("User:", info.getUser()).
+        __("Started on:", Times.format(info.getStartTime())).
+        __("Elasped: ", org.apache.hadoop.util.StringUtils.formatTime(
           info.getElapsedTime() ));
     render(InfoPage.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java
index 7fde95b..7a3e2b3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppView.java
@@ -25,14 +25,14 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 public class AppView extends TwoColumnLayout {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "jobs");
     set(initID(DATATABLES, "jobs"), jobsTableInit());
     setTableStyles(html, "jobs");
   }
 
-  protected void commonPreHead(Page.HTML<_> html) {
+  protected void commonPreHead(Page.HTML<__> html) {
     set(ACCORDION_ID, "nav");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
index 98a2ce1..76ef6bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
@@ -30,10 +30,10 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -56,21 +56,21 @@ public class ConfBlock extends HtmlBlock {
     String jid = $(JOB_ID);
     if (jid.isEmpty()) {
       html.
-        p()._("Sorry, can't do anything without a JobID.")._();
+        p().__("Sorry, can't do anything without a JobID.").__();
       return;
     }
     JobId jobID = MRApps.toJobID(jid);
     Job job = appContext.getJob(jobID);
     if (job == null) {
       html.
-        p()._("Sorry, ", jid, " not found.")._();
+        p().__("Sorry, ", jid, " not found.").__();
       return;
     }
     Path confPath = job.getConfFile();
     try {
       ConfInfo info = new ConfInfo(job);
 
-      html.div().a("/jobhistory/downloadconf/" + jid, confPath.toString())._();
+      html.div().a("/jobhistory/downloadconf/" + jid, confPath.toString()).__();
       TBODY<TABLE<Hamlet>> tbody = html.
         // Tasks table
       table("#conf").
@@ -79,8 +79,8 @@ public class ConfBlock extends HtmlBlock {
             th(_TH, "key").
             th(_TH, "value").
             th(_TH, "source chain").
-          _().
-        _().
+              __().
+              __().
       tbody();
       for (ConfEntryInfo entry : info.getProperties()) {
         StringBuffer buffer = new StringBuffer();
@@ -100,20 +100,20 @@ public class ConfBlock extends HtmlBlock {
             td(entry.getName()).
             td(entry.getValue()).
             td(buffer.toString()).
-          _();
+            __();
       }
-      tbody._().
+      tbody.__().
       tfoot().
         tr().
-          th().input("search_init").$type(InputType.text).$name("key").$value("key")._()._().
-          th().input("search_init").$type(InputType.text).$name("value").$value("value")._()._().
-          th().input("search_init").$type(InputType.text).$name("source chain").$value("source chain")._()._().
-          _().
-        _().
-      _();
+          th().input("search_init").$type(InputType.text).$name("key").$value("key").__().__().
+          th().input("search_init").$type(InputType.text).$name("value").$value("value").__().__().
+          th().input("search_init").$type(InputType.text).$name("source chain").$value("source chain").__().__().
+          __().
+          __().
+          __();
     } catch(IOException e) {
       LOG.error("Error while reading "+confPath, e);
-      html.p()._("Sorry got an error while reading conf file. ",confPath);
+      html.p().__("Sorry got an error while reading conf file. ", confPath);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
index 568658e..4b9e6f4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersBlock.java
@@ -34,13 +34,13 @@ import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.THEAD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -60,12 +60,12 @@ public class CountersBlock extends HtmlBlock {
   @Override protected void render(Block html) {
     if (job == null) {
       html.
-        p()._("Sorry, no counters for nonexistent", $(JOB_ID, "job"))._();
+        p().__("Sorry, no counters for nonexistent", $(JOB_ID, "job")).__();
       return;
     }
     if (!$(TASK_ID).isEmpty() && task == null) {
       html.
-        p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
+        p().__("Sorry, no counters for nonexistent", $(TASK_ID, "task")).__();
       return;
     }
     
@@ -75,7 +75,7 @@ public class CountersBlock extends HtmlBlock {
         type = $(JOB_ID, "the job");
       }
       html.
-        p()._("Sorry it looks like ",type," has no counters.")._();
+        p().__("Sorry it looks like ", type, " has no counters.").__();
       return;
     }
     
@@ -97,7 +97,7 @@ public class CountersBlock extends HtmlBlock {
         thead().
           tr().
             th(".group.ui-state-default", "Counter Group").
-            th(".ui-state-default", "Counters")._()._().
+            th(".ui-state-default", "Counters").__().__().
         tbody();
     for (CounterGroup g : total) {
       CounterGroup mg = map == null ? null : map.getGroup(g.getName());
@@ -109,7 +109,7 @@ public class CountersBlock extends HtmlBlock {
       TR<THEAD<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupHeadRow = tbody.
         tr().
           th().$title(g.getName()).$class("ui-state-default").
-            _(fixGroupDisplayName(g.getDisplayName()))._().
+          __(fixGroupDisplayName(g.getDisplayName())).__().
           td().$class(C_TABLE).
             table(".dt-counters").$id(job.getID()+"."+g.getName()).
               thead().
@@ -120,20 +120,20 @@ public class CountersBlock extends HtmlBlock {
       }
       // Ditto
       TBODY<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>> group = groupHeadRow.
-            th(map == null ? "Value" : "Total")._()._().
+            th(map == null ? "Value" : "Total").__().__().
         tbody();
       for (Counter counter : g) {
         // Ditto
         TR<TBODY<TABLE<TD<TR<TBODY<TABLE<DIV<Hamlet>>>>>>>> groupRow = group.
           tr();
           if (task == null && mg == null && rg == null) {
-            groupRow.td().$title(counter.getName())._(counter.getDisplayName()).
-            _();
+            groupRow.td().$title(counter.getName()).__(counter.getDisplayName()).
+                __();
           } else {
             groupRow.td().$title(counter.getName()).
               a(url(urlBase,urlId,g.getName(), 
                   counter.getName()), counter.getDisplayName()).
-            _();
+                __();
           }
         if (map != null) {
           Counter mc = mg == null ? null : mg.findCounter(counter.getName());
@@ -142,11 +142,11 @@ public class CountersBlock extends HtmlBlock {
             td(mc == null ? "0" : String.format("%,d", mc.getValue())).
             td(rc == null ? "0" : String.format("%,d", rc.getValue()));
         }
-        groupRow.td(String.format("%,d", counter.getValue()))._();
+        groupRow.td(String.format("%,d", counter.getValue())).__();
       }
-      group._()._()._()._();
+      group.__().__().__().__();
     }
-    tbody._()._()._();
+    tbody.__().__().__();
   }
 
   private void getCounters(AppContext ctx) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
index d7afcd8..e780907 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/CountersPage.java
@@ -25,7 +25,7 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 public class CountersPage extends AppView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     String tid = $(TASK_ID);
@@ -39,7 +39,7 @@ public class CountersPage extends AppView {
         "{bJQueryUI:true, sDom:'t', iDisplayLength:-1}");
   }
 
-  @Override protected void postHead(Page.HTML<_> html) {
+  @Override protected void postHead(Page.HTML<__> html) {
     html.
       style("#counters, .dt-counters { table-layout: fixed }",
             "#counters th { overflow: hidden; vertical-align: middle }",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/InfoPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/InfoPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/InfoPage.java
index 5163a01..3dd64f5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/InfoPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/InfoPage.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
 public class InfoPage extends AppView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setTitle("About the Application Master");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
index a599870..77ea55e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
@@ -30,7 +30,6 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._TH;
 import java.util.Date;
 import java.util.List;
 
-import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
@@ -41,9 +40,9 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -60,14 +59,14 @@ public class JobBlock extends HtmlBlock {
     String jid = $(JOB_ID);
     if (jid.isEmpty()) {
       html.
-        p()._("Sorry, can't do anything without a JobID.")._();
+        p().__("Sorry, can't do anything without a JobID.").__();
       return;
     }
     JobId jobID = MRApps.toJobID(jid);
     Job job = appContext.getJob(jobID);
     if (job == null) {
       html.
-        p()._("Sorry, ", jid, " not found.")._();
+        p().__("Sorry, ", jid, " not found.").__();
       return;
     }
 
@@ -77,15 +76,15 @@ public class JobBlock extends HtmlBlock {
 
     JobInfo jinfo = new JobInfo(job, true);
     info("Job Overview").
-        _("Job Name:", jinfo.getName()).
-        _("User Name:", jinfo.getUserName()).
-        _("Queue Name:", jinfo.getQueueName()).
-        _("State:", jinfo.getState()).
-        _("Uberized:", jinfo.isUberized()).
-        _("Started:", new Date(jinfo.getStartTime())).
-        _("Elapsed:", StringUtils.formatTime(jinfo.getElapsedTime()));
+        __("Job Name:", jinfo.getName()).
+        __("User Name:", jinfo.getUserName()).
+        __("Queue Name:", jinfo.getQueueName()).
+        __("State:", jinfo.getState()).
+        __("Uberized:", jinfo.isUberized()).
+        __("Started:", new Date(jinfo.getStartTime())).
+        __("Elapsed:", StringUtils.formatTime(jinfo.getElapsedTime()));
     DIV<Hamlet> div = html.
-      _(InfoBlock.class).
+        __(InfoBlock.class).
       div(_INFO_WRAP);
 
     // MRAppMasters Table
@@ -93,13 +92,13 @@ public class JobBlock extends HtmlBlock {
     table.
       tr().
       th(amString).
-      _().
+        __().
       tr().
       th(_TH, "Attempt Number").
       th(_TH, "Start Time").
       th(_TH, "Node").
       th(_TH, "Logs").
-      _();
+        __();
     for (AMInfo amInfo : amInfos) {
       AMAttemptInfo attempt = new AMAttemptInfo(amInfo,
           jinfo.getId(), jinfo.getUserName());
@@ -109,14 +108,14 @@ public class JobBlock extends HtmlBlock {
         td(new Date(attempt.getStartTime()).toString()).
         td().a(".nodelink", url(MRWebAppUtil.getYARNWebappScheme(),
             attempt.getNodeHttpAddress()),
-            attempt.getNodeHttpAddress())._().
+            attempt.getNodeHttpAddress()).__().
         td().a(".logslink", url(attempt.getLogsLink()), 
-            "logs")._().
-        _();
+            "logs").__().
+          __();
     }
 
-    table._();
-    div._();
+    table.__();
+    div.__();
 
     html.div(_INFO_WRAP).        
       // Tasks table
@@ -127,30 +126,30 @@ public class JobBlock extends HtmlBlock {
             th(_TH, "Total").
             th(_TH, "Pending").
             th(_TH, "Running").
-            th(_TH, "Complete")._().
+            th(_TH, "Complete").__().
           tr(_ODD).
             th("Map").
             td().
               div(_PROGRESSBAR).
                 $title(join(jinfo.getMapProgressPercent(), '%')). // tooltip
                 div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", jinfo.getMapProgressPercent(), '%'))._()._()._().
-            td().a(url("tasks", jid, "m", "ALL"),String.valueOf(jinfo.getMapsTotal()))._().
-            td().a(url("tasks", jid, "m", "PENDING"),String.valueOf(jinfo.getMapsPending()))._().
-            td().a(url("tasks", jid, "m", "RUNNING"),String.valueOf(jinfo.getMapsRunning()))._().
-            td().a(url("tasks", jid, "m", "COMPLETED"),String.valueOf(jinfo.getMapsCompleted()))._()._().
+                  $style(join("width:", jinfo.getMapProgressPercent(), '%')).__().__().__().
+            td().a(url("tasks", jid, "m", "ALL"), String.valueOf(jinfo.getMapsTotal())).__().
+            td().a(url("tasks", jid, "m", "PENDING"), String.valueOf(jinfo.getMapsPending())).__().
+            td().a(url("tasks", jid, "m", "RUNNING"), String.valueOf(jinfo.getMapsRunning())).__().
+            td().a(url("tasks", jid, "m", "COMPLETED"), String.valueOf(jinfo.getMapsCompleted())).__().__().
           tr(_EVEN).
             th("Reduce").
             td().
               div(_PROGRESSBAR).
                 $title(join(jinfo.getReduceProgressPercent(), '%')). // tooltip
                 div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", jinfo.getReduceProgressPercent(), '%'))._()._()._().
-            td().a(url("tasks", jid, "r", "ALL"),String.valueOf(jinfo.getReducesTotal()))._().
-            td().a(url("tasks", jid, "r", "PENDING"),String.valueOf(jinfo.getReducesPending()))._().
-            td().a(url("tasks", jid, "r", "RUNNING"),String.valueOf(jinfo.getReducesRunning()))._().
-            td().a(url("tasks", jid, "r", "COMPLETED"),String.valueOf(jinfo.getReducesCompleted()))._()._()
-          ._().
+                  $style(join("width:", jinfo.getReduceProgressPercent(), '%')).__().__().__().
+            td().a(url("tasks", jid, "r", "ALL"), String.valueOf(jinfo.getReducesTotal())).__().
+            td().a(url("tasks", jid, "r", "PENDING"), String.valueOf(jinfo.getReducesPending())).__().
+            td().a(url("tasks", jid, "r", "RUNNING"), String.valueOf(jinfo.getReducesRunning())).__().
+            td().a(url("tasks", jid, "r", "COMPLETED"), String.valueOf(jinfo.getReducesCompleted())).__().__()
+          .__().
         // Attempts table
         table("#job").
         tr().
@@ -159,45 +158,45 @@ public class JobBlock extends HtmlBlock {
           th(_TH, "Running").
           th(_TH, "Failed").
           th(_TH, "Killed").
-          th(_TH, "Successful")._().
+          th(_TH, "Successful").__().
         tr(_ODD).
           th("Maps").
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.NEW.toString()),
-              String.valueOf(jinfo.getNewMapAttempts()))._().
+              String.valueOf(jinfo.getNewMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.RUNNING.toString()),
-              String.valueOf(jinfo.getRunningMapAttempts()))._().
+              String.valueOf(jinfo.getRunningMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.FAILED.toString()),
-              String.valueOf(jinfo.getFailedMapAttempts()))._().
+              String.valueOf(jinfo.getFailedMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.KILLED.toString()),
-              String.valueOf(jinfo.getKilledMapAttempts()))._().
+              String.valueOf(jinfo.getKilledMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.SUCCESSFUL.toString()),
-              String.valueOf(jinfo.getSuccessfulMapAttempts()))._().
-        _().
+              String.valueOf(jinfo.getSuccessfulMapAttempts())).__().
+        __().
         tr(_EVEN).
           th("Reduces").
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.NEW.toString()),
-              String.valueOf(jinfo.getNewReduceAttempts()))._().
+              String.valueOf(jinfo.getNewReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.RUNNING.toString()),
-              String.valueOf(jinfo.getRunningReduceAttempts()))._().
+              String.valueOf(jinfo.getRunningReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.FAILED.toString()),
-              String.valueOf(jinfo.getFailedReduceAttempts()))._().
+              String.valueOf(jinfo.getFailedReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.KILLED.toString()),
-              String.valueOf(jinfo.getKilledReduceAttempts()))._().
+              String.valueOf(jinfo.getKilledReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.SUCCESSFUL.toString()),
-              String.valueOf(jinfo.getSuccessfulReduceAttempts()))._().
-         _().
-       _().
-     _();
+              String.valueOf(jinfo.getSuccessfulReduceAttempts())).__().
+        __().
+        __().
+        __();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java
index 983859e..4d6a3e2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java
@@ -27,7 +27,6 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.postInitID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
 
-import org.apache.hadoop.mapreduce.v2.app.webapp.ConfBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 
 /**
@@ -39,7 +38,7 @@ public class JobConfPage extends AppView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String jobID = $(JOB_ID);
     set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
         : join("Configuration for MapReduce Job ", $(JOB_ID)));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobPage.java
index 00f4750..6508fb8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobPage.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.webapp.SubView;
 
 public class JobPage extends AppView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String jobID = $(JOB_ID);
     set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
                : join("MapReduce Job ", $(JOB_ID)));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java
index 720219e..ff4bc00 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java
@@ -25,9 +25,9 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -53,34 +53,34 @@ public class JobsBlock extends HtmlBlock {
             th("Maps Completed").
             th("Reduce Progress").
             th("Reduces Total").
-            th("Reduces Completed")._()._().
+            th("Reduces Completed").__().__().
         tbody();
     for (Job j : appContext.getAllJobs().values()) {
       JobInfo job = new JobInfo(j, false);
       tbody.
         tr().
           td().
-            span().$title(String.valueOf(job.getId()))._(). // for sorting
-            a(url("job", job.getId()), job.getId())._().
+            span().$title(String.valueOf(job.getId())).__(). // for sorting
+            a(url("job", job.getId()), job.getId()).__().
           td(job.getName()).
           td(job.getState()).
           td().
-            span().$title(job.getMapProgressPercent())._(). // for sorting
+            span().$title(job.getMapProgressPercent()).__(). // for sorting
             div(_PROGRESSBAR).
               $title(join(job.getMapProgressPercent(), '%')). // tooltip
               div(_PROGRESSBAR_VALUE).
-                $style(join("width:", job.getMapProgressPercent(), '%'))._()._()._().
+                $style(join("width:", job.getMapProgressPercent(), '%')).__().__().__().
           td(String.valueOf(job.getMapsTotal())).
           td(String.valueOf(job.getMapsCompleted())).
           td().
-            span().$title(job.getReduceProgressPercent())._(). // for sorting
+            span().$title(job.getReduceProgressPercent()).__(). // for sorting
             div(_PROGRESSBAR).
               $title(join(job.getReduceProgressPercent(), '%')). // tooltip
               div(_PROGRESSBAR_VALUE).
-                $style(join("width:", job.getReduceProgressPercent(), '%'))._()._()._().
+                $style(join("width:", job.getReduceProgressPercent(), '%')).__().__().__().
           td(String.valueOf(job.getReducesTotal())).
-          td(String.valueOf(job.getReducesCompleted()))._();
+          td(String.valueOf(job.getReducesCompleted())).__();
     }
-    tbody._()._();
+    tbody.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
index 4eed7e3..58e1a43 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
@@ -25,8 +25,8 @@ import java.util.List;
 import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -42,13 +42,13 @@ public class NavBlock extends HtmlBlock {
       div("#nav").
         h3("Cluster").
         ul().
-          li().a(url(rmweb, "cluster", "cluster"), "About")._().
-          li().a(url(rmweb, "cluster", "apps"), "Applications")._().
-          li().a(url(rmweb, "cluster", "scheduler"), "Scheduler")._()._().
+          li().a(url(rmweb, "cluster", "cluster"), "About").__().
+          li().a(url(rmweb, "cluster", "apps"), "Applications").__().
+          li().a(url(rmweb, "cluster", "scheduler"), "Scheduler").__().__().
         h3("Application").
         ul().
-          li().a(url("app/info"), "About")._().
-          li().a(url("app"), "Jobs")._()._();
+          li().a(url("app/info"), "About").__().
+          li().a(url("app"), "Jobs").__().__();
     if (app.getJob() != null) {
       String jobid = MRApps.toString(app.getJob().getID());
       List<AMInfo> amInfos = app.getJob().getAMInfos();
@@ -58,31 +58,31 @@ public class NavBlock extends HtmlBlock {
       nav.
         h3("Job").
         ul().
-          li().a(url("job", jobid), "Overview")._().
-          li().a(url("jobcounters", jobid), "Counters")._().
-          li().a(url("conf", jobid), "Configuration")._().
-          li().a(url("tasks", jobid, "m"), "Map tasks")._().
-          li().a(url("tasks", jobid, "r"), "Reduce tasks")._().
+          li().a(url("job", jobid), "Overview").__().
+          li().a(url("jobcounters", jobid), "Counters").__().
+          li().a(url("conf", jobid), "Configuration").__().
+          li().a(url("tasks", jobid, "m"), "Map tasks").__().
+          li().a(url("tasks", jobid, "r"), "Reduce tasks").__().
           li().a(".logslink", url(MRWebAppUtil.getYARNWebappScheme(),
               nodeHttpAddress, "node",
               "containerlogs", thisAmInfo.getContainerId().toString(), 
               app.getJob().getUserName()), 
-              "AM Logs")._()._();
+              "AM Logs").__().__();
       if (app.getTask() != null) {
         String taskid = MRApps.toString(app.getTask().getID());
         nav.
           h3("Task").
           ul().
-            li().a(url("task", taskid), "Task Overview")._().
-            li().a(url("taskcounters", taskid), "Counters")._()._();
+            li().a(url("task", taskid), "Task Overview").__().
+            li().a(url("taskcounters", taskid), "Counters").__().__();
       }
     }
     nav.
       h3("Tools").
       ul().
-        li().a("/conf", "Configuration")._().
-        li().a("/logs", "Local logs")._().
-        li().a("/stacks", "Server stacks")._().
-        li().a("/jmx?qry=Hadoop:*", "Server metrics")._()._()._();
+        li().a("/conf", "Configuration").__().
+        li().a("/logs", "Local logs").__().
+        li().a("/stacks", "Server stacks").__().
+        li().a("/jmx?qry=Hadoop:*", "Server metrics").__().__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
index c4311e9..02fb226 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterBlock.java
@@ -39,11 +39,11 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -62,12 +62,12 @@ public class SingleCounterBlock extends HtmlBlock {
   @Override protected void render(Block html) {
     if (job == null) {
       html.
-        p()._("Sorry, no counters for nonexistent", $(JOB_ID, "job"))._();
+        p().__("Sorry, no counters for nonexistent", $(JOB_ID, "job")).__();
       return;
     }
     if (!$(TASK_ID).isEmpty() && task == null) {
       html.
-        p()._("Sorry, no counters for nonexistent", $(TASK_ID, "task"))._();
+        p().__("Sorry, no counters for nonexistent", $(TASK_ID, "task")).__();
       return;
     }
     
@@ -79,7 +79,7 @@ public class SingleCounterBlock extends HtmlBlock {
         thead().
           tr().
             th(".ui-state-default", columnType).
-            th(".ui-state-default", "Value")._()._().
+            th(".ui-state-default", "Value").__().__().
           tbody();
     for (Map.Entry<String, Long> entry : values.entrySet()) {
       TR<TBODY<TABLE<DIV<Hamlet>>>> row = tbody.tr();
@@ -87,16 +87,16 @@ public class SingleCounterBlock extends HtmlBlock {
       String val = entry.getValue().toString();
       if(task != null) {
         row.td(id);
-        row.td().br().$title(val)._()._(val)._();
+        row.td().br().$title(val).__().__(val).__();
       } else {
         row.td().a(url("singletaskcounter",entry.getKey(),
-            $(COUNTER_GROUP), $(COUNTER_NAME)), id)._();
-        row.td().br().$title(val)._().a(url("singletaskcounter",entry.getKey(),
-            $(COUNTER_GROUP), $(COUNTER_NAME)), val)._();
+            $(COUNTER_GROUP), $(COUNTER_NAME)), id).__();
+        row.td().br().$title(val).__().a(url("singletaskcounter", entry.getKey(),
+            $(COUNTER_GROUP), $(COUNTER_NAME)), val).__();
       }
-      row._();
+      row.__();
     }
-    tbody._()._()._();
+    tbody.__().__().__();
   }
 
   private void populateMembers(AppContext ctx) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
index 729b5a8..6fc1f82 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/SingleCounterPage.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app.webapp;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
-import org.apache.hadoop.mapreduce.v2.app.webapp.SingleCounterBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 
 /**
@@ -33,7 +32,7 @@ public class SingleCounterPage extends AppView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     String tid = $(TASK_ID);
     String activeNav = "3";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
index 01c5b0d..bd7f7a9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
@@ -38,11 +38,11 @@ import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.THEAD;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TR;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -100,7 +100,7 @@ public class TaskPage extends AppView {
             .append("  }\n")
             .append("}\n");
 
-        html.script().$type("text/javascript")._(script.toString())._();
+        html.script().$type("text/javascript").__(script.toString()).__();
       }
 
       TR<THEAD<TABLE<Hamlet>>> tr = html.table("#attempts").thead().tr();
@@ -118,7 +118,7 @@ public class TaskPage extends AppView {
         tr.th(".actions", "Actions");
       }
 
-      TBODY<TABLE<Hamlet>> tbody = tr._()._().tbody();
+      TBODY<TABLE<Hamlet>> tbody = tr.__().__().tbody();
       // Write all the data into a JavaScript array of arrays for JQuery
       // DataTables to display
       StringBuilder attemptsTableData = new StringBuilder("[\n");
@@ -178,9 +178,9 @@ public class TaskPage extends AppView {
       }
       attemptsTableData.append("]");
       html.script().$type("text/javascript").
-      _("var attemptsTableData=" + attemptsTableData)._();
+          __("var attemptsTableData=" + attemptsTableData).__();
 
-      tbody._()._();
+      tbody.__().__();
 
     }
 
@@ -197,7 +197,7 @@ public class TaskPage extends AppView {
     }
   }
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
 
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:3}");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
index 78338ec..8d92dd3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -65,7 +65,7 @@ public class TasksBlock extends HtmlBlock {
             th("State").
             th("Start Time").
             th("Finish Time").
-            th("Elapsed Time")._()._().
+            th("Elapsed Time").__().__().
         tbody();
     StringBuilder tasksTableData = new StringBuilder("[\n");
 
@@ -117,8 +117,8 @@ public class TasksBlock extends HtmlBlock {
     }
     tasksTableData.append("]");
     html.script().$type("text/javascript").
-    _("var tasksTableData=" + tasksTableData)._();
+        __("var tasksTableData=" + tasksTableData).__();
 
-    tbody._()._();
+    tbody.__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
index e2f12dc..8fce395 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksPage.java
@@ -24,7 +24,7 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 public class TasksPage extends AppView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "tasks");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:2}");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java
index f607599..d544c6b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 
-import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -36,7 +35,7 @@ public class HsAboutPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     //override the nav config from commonPReHead
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
@@ -49,9 +48,9 @@ public class HsAboutPage extends HsView {
   @Override protected Class<? extends SubView> content() {
     HistoryInfo info = new HistoryInfo();
     info("History Server").
-      _("BuildVersion", info.getHadoopBuildVersion()
+        __("BuildVersion", info.getHadoopBuildVersion()
         + " on " + info.getHadoopVersionBuiltOn()).
-      _("History Server started on", Times.format(info.getStartedOn()));
+        __("History Server started on", Times.format(info.getStartedOn()));
     return InfoBlock.class;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java
index 8431e22..c08ee5c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java
@@ -39,7 +39,7 @@ public class HsConfPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String jobID = $(JOB_ID);
     set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
         : join("Configuration for MapReduce Job ", $(JOB_ID)));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
index e70a668..1632a97 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsCountersPage.java
@@ -32,7 +32,7 @@ public class HsCountersPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setActiveNavColumnForTask();
     set(DATATABLES_SELECTOR, "#counters .dt-counters");
@@ -44,7 +44,7 @@ public class HsCountersPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.yarn.webapp.view.TwoColumnLayout#postHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void postHead(Page.HTML<_> html) {
+  @Override protected void postHead(Page.HTML<__> html) {
     html.
       style("#counters, .dt-counters { table-layout: fixed }",
             "#counters th { overflow: hidden; vertical-align: middle }",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
index 0d5b03a..18040f0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
@@ -43,9 +43,9 @@ import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -69,38 +69,38 @@ public class HsJobBlock extends HtmlBlock {
     String jid = $(JOB_ID);
     if (jid.isEmpty()) {
       html.
-        p()._("Sorry, can't do anything without a JobID.")._();
+        p().__("Sorry, can't do anything without a JobID.").__();
       return;
     }
     JobId jobID = MRApps.toJobID(jid);
     Job j = appContext.getJob(jobID);
     if (j == null) {
-      html.p()._("Sorry, ", jid, " not found.")._();
+      html.p().__("Sorry, ", jid, " not found.").__();
       return;
     }
     if(j instanceof UnparsedJob) {
       final int taskCount = j.getTotalMaps() + j.getTotalReduces();
       UnparsedJob oversizedJob = (UnparsedJob) j;
-      html.p()._("The job has a total of " + taskCount + " tasks. ")
-          ._("Any job larger than " + oversizedJob.getMaxTasksAllowed() +
-              " will not be loaded.")._();
-      html.p()._("You can either use the CLI tool: 'mapred job -history'"
+      html.p().__("The job has a total of " + taskCount + " tasks. ")
+          .__("Any job larger than " + oversizedJob.getMaxTasksAllowed() +
+              " will not be loaded.").__();
+      html.p().__("You can either use the CLI tool: 'mapred job -history'"
           + " to view large jobs or adjust the property " +
-          JHAdminConfig.MR_HS_LOADED_JOBS_TASKS_MAX + ".")._();
+          JHAdminConfig.MR_HS_LOADED_JOBS_TASKS_MAX + ".").__();
       return;
     }
     List<AMInfo> amInfos = j.getAMInfos();
     JobInfo job = new JobInfo(j);
     ResponseInfo infoBlock = info("Job Overview").
-        _("Job Name:", job.getName()).
-        _("User Name:", job.getUserName()).
-        _("Queue:", job.getQueueName()).
-        _("State:", job.getState()).
-        _("Uberized:", job.isUber()).
-        _("Submitted:", new Date(job.getSubmitTime())).
-        _("Started:", job.getStartTimeStr()).
-        _("Finished:", new Date(job.getFinishTime())).
-        _("Elapsed:", StringUtils.formatTime(
+        __("Job Name:", job.getName()).
+        __("User Name:", job.getUserName()).
+        __("Queue:", job.getQueueName()).
+        __("State:", job.getState()).
+        __("Uberized:", job.isUber()).
+        __("Submitted:", new Date(job.getSubmitTime())).
+        __("Started:", job.getStartTimeStr()).
+        __("Finished:", new Date(job.getFinishTime())).
+        __("Elapsed:", StringUtils.formatTime(
             Times.elapsed(job.getStartTime(), job.getFinishTime(), false)));
     
     String amString =
@@ -117,19 +117,19 @@ public class HsJobBlock extends HtmlBlock {
     }
 
     if(job.getNumMaps() > 0) {
-      infoBlock._("Average Map Time", StringUtils.formatTime(job.getAvgMapTime()));
+      infoBlock.__("Average Map Time", StringUtils.formatTime(job.getAvgMapTime()));
     }
     if(job.getNumReduces() > 0) {
-      infoBlock._("Average Shuffle Time", StringUtils.formatTime(job.getAvgShuffleTime()));
-      infoBlock._("Average Merge Time", StringUtils.formatTime(job.getAvgMergeTime()));
-      infoBlock._("Average Reduce Time", StringUtils.formatTime(job.getAvgReduceTime()));
+      infoBlock.__("Average Shuffle Time", StringUtils.formatTime(job.getAvgShuffleTime()));
+      infoBlock.__("Average Merge Time", StringUtils.formatTime(job.getAvgMergeTime()));
+      infoBlock.__("Average Reduce Time", StringUtils.formatTime(job.getAvgReduceTime()));
     }
 
     for (ConfEntryInfo entry : job.getAcls()) {
-      infoBlock._("ACL "+entry.getName()+":", entry.getValue());
+      infoBlock.__("ACL "+entry.getName()+":", entry.getValue());
     }
     DIV<Hamlet> div = html.
-      _(InfoBlock.class).
+        __(InfoBlock.class).
       div(_INFO_WRAP);
     
       // MRAppMasters Table
@@ -137,13 +137,13 @@ public class HsJobBlock extends HtmlBlock {
         table.
           tr().
             th(amString).
-          _().
+            __().
           tr().
             th(_TH, "Attempt Number").
             th(_TH, "Start Time").
             th(_TH, "Node").
             th(_TH, "Logs").
-            _();
+            __();
         boolean odd = false;
           for (AMInfo amInfo : amInfos) {
             AMAttemptInfo attempt = new AMAttemptInfo(amInfo,
@@ -153,13 +153,13 @@ public class HsJobBlock extends HtmlBlock {
               td(new Date(attempt.getStartTime()).toString()).
               td().a(".nodelink", url(MRWebAppUtil.getYARNWebappScheme(),
                   attempt.getNodeHttpAddress()),
-                  attempt.getNodeHttpAddress())._().
+                  attempt.getNodeHttpAddress()).__().
               td().a(".logslink", url(attempt.getLogsLink()),
-                      "logs")._().
-            _();
+                      "logs").__().
+                __();
           }
-          table._();
-          div._();
+          table.__();
+          div.__();
           
         
         html.div(_INFO_WRAP).        
@@ -169,18 +169,18 @@ public class HsJobBlock extends HtmlBlock {
           tr().
             th(_TH, "Task Type").
             th(_TH, "Total").
-            th(_TH, "Complete")._().
+            th(_TH, "Complete").__().
           tr(_ODD).
             th().
-              a(url("tasks", jid, "m"), "Map")._().
+              a(url("tasks", jid, "m"), "Map").__().
             td(String.valueOf(String.valueOf(job.getMapsTotal()))).
-            td(String.valueOf(String.valueOf(job.getMapsCompleted())))._().
+            td(String.valueOf(String.valueOf(job.getMapsCompleted()))).__().
           tr(_EVEN).
             th().
-              a(url("tasks", jid, "r"), "Reduce")._().
+              a(url("tasks", jid, "r"), "Reduce").__().
             td(String.valueOf(String.valueOf(job.getReducesTotal()))).
-            td(String.valueOf(String.valueOf(job.getReducesCompleted())))._()
-          ._().
+            td(String.valueOf(String.valueOf(job.getReducesCompleted()))).__()
+          .__().
 
         // Attempts table
         table("#job").
@@ -188,33 +188,33 @@ public class HsJobBlock extends HtmlBlock {
           th(_TH, "Attempt Type").
           th(_TH, "Failed").
           th(_TH, "Killed").
-          th(_TH, "Successful")._().
+          th(_TH, "Successful").__().
         tr(_ODD).
           th("Maps").
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(job.getFailedMapAttempts()))._().
+              String.valueOf(job.getFailedMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(job.getKilledMapAttempts()))._().
+              String.valueOf(job.getKilledMapAttempts())).__().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(job.getSuccessfulMapAttempts()))._().
-        _().
+              String.valueOf(job.getSuccessfulMapAttempts())).__().
+            __().
         tr(_EVEN).
           th("Reduces").
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(job.getFailedReduceAttempts()))._().
+              String.valueOf(job.getFailedReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(job.getKilledReduceAttempts()))._().
+              String.valueOf(job.getKilledReduceAttempts())).__().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(job.getSuccessfulReduceAttempts()))._().
-         _().
-       _().
-     _();
+              String.valueOf(job.getSuccessfulReduceAttempts())).__().
+            __().
+            __().
+            __();
   }
 
   static String addTaskLinks(String text) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java
index 4c81a13..f40c878 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobPage.java
@@ -34,7 +34,7 @@ public class HsJobPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     String jobID = $(JOB_ID);
     set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
                : join("MapReduce Job ", $(JOB_ID)));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
index b234ca3..ef563f6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
@@ -27,10 +27,10 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -69,7 +69,7 @@ public class HsJobsBlock extends HtmlBlock {
             th("Maps Completed").
             th("Reduces Total").
             th("Reduces Completed").
-            th("Elapsed Time")._()._().
+            th("Elapsed Time").__().__().
         tbody();
     LOG.info("Getting list of all Jobs.");
     // Write all the data into a JavaScript array of arrays for JQuery
@@ -105,38 +105,38 @@ public class HsJobsBlock extends HtmlBlock {
     }
     jobsTableData.append("]");
     html.script().$type("text/javascript").
-    _("var jobsTableData=" + jobsTableData)._();
-    tbody._().
+        __("var jobsTableData=" + jobsTableData).__();
+    tbody.__().
     tfoot().
       tr().
         th().input("search_init").$type(InputType.text)
-          .$name("submit_time").$value("Submit Time")._()._().
+          .$name("submit_time").$value("Submit Time").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("start_time").$value("Start Time")._()._().
+          .$name("start_time").$value("Start Time").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("finish_time").$value("Finish Time")._()._().
+          .$name("finish_time").$value("Finish Time").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("job_id").$value("Job ID")._()._().
+          .$name("job_id").$value("Job ID").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("name").$value("Name")._()._().
+          .$name("name").$value("Name").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("user").$value("User")._()._().
+          .$name("user").$value("User").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("queue").$value("Queue")._()._().
+          .$name("queue").$value("Queue").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("state").$value("State")._()._().
+          .$name("state").$value("State").__().__().
         th().input("search_init").$type(InputType.text)
-          .$name("maps_total").$value("Maps Total")._()._().
+          .$name("maps_total").$value("Maps Total").__().__().
         th().input("search_init").$type(InputType.text).
-          $name("maps_completed").$value("Maps Completed")._()._().
+          $name("maps_completed").$value("Maps Completed").__().__().
         th().input("search_init").$type(InputType.text).
-          $name("reduces_total").$value("Reduces Total")._()._().
+          $name("reduces_total").$value("Reduces Total").__().__().
         th().input("search_init").$type(InputType.text).
-          $name("reduces_completed").$value("Reduces Completed")._()._().
+          $name("reduces_completed").$value("Reduces Completed").__().__().
         th().input("search_init").$type(InputType.text).
-          $name("elapsed_time").$value("Elapsed Time")._()._().
-        _().
-      _().
-    _();
+          $name("elapsed_time").$value("Elapsed Time").__().__().
+        __().
+        __().
+        __();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
index f483dc9..2bee3ba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsLogsPage.java
@@ -26,7 +26,7 @@ public class HsLogsPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setActiveNavColumnForTask();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
index 7e49d52..9ef5a0f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -43,33 +43,33 @@ public class HsNavBlock extends HtmlBlock {
       div("#nav").
       h3("Application").
         ul().
-          li().a(url("about"), "About")._().
-          li().a(url("app"), "Jobs")._()._();
+          li().a(url("about"), "About").__().
+          li().a(url("app"), "Jobs").__().__();
     if (app.getJob() != null) {
       String jobid = MRApps.toString(app.getJob().getID());
       nav.
         h3("Job").
         ul().
-          li().a(url("job", jobid), "Overview")._().
-          li().a(url("jobcounters", jobid), "Counters")._().
-          li().a(url("conf", jobid), "Configuration")._().
-          li().a(url("tasks", jobid, "m"), "Map tasks")._().
-          li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
+          li().a(url("job", jobid), "Overview").__().
+          li().a(url("jobcounters", jobid), "Counters").__().
+          li().a(url("conf", jobid), "Configuration").__().
+          li().a(url("tasks", jobid, "m"), "Map tasks").__().
+          li().a(url("tasks", jobid, "r"), "Reduce tasks").__().__();
       if (app.getTask() != null) {
         String taskid = MRApps.toString(app.getTask().getID());
         nav.
           h3("Task").
           ul().
-            li().a(url("task", taskid), "Task Overview")._().
-            li().a(url("taskcounters", taskid), "Counters")._()._();
+            li().a(url("task", taskid), "Task Overview").__().
+            li().a(url("taskcounters", taskid), "Counters").__().__();
       }
     }
     nav.
       h3("Tools").
         ul().
-          li().a("/conf", "Configuration")._().
-          li().a("/logs", "Local logs")._().
-          li().a("/stacks", "Server stacks")._().
-          li().a("/jmx?qry=Hadoop:*", "Server metrics")._()._()._();
+          li().a("/conf", "Configuration").__().
+          li().a("/logs", "Local logs").__().
+          li().a("/stacks", "Server stacks").__().
+          li().a("/jmx?qry=Hadoop:*", "Server metrics").__().__().__();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c6fa5c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
index 5f97b8f..bc2c2c8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsSingleCounterPage.java
@@ -32,7 +32,7 @@ public class HsSingleCounterPage extends HsView {
    * (non-Javadoc)
    * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
    */
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
     setActiveNavColumnForTask();
     set(DATATABLES_ID, "singleCounter");


---------------------------------------------------------------------
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: HDFS-10629. Federation Roter. Contributed by Jason Kace and Inigo Goiri.

Posted by in...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
new file mode 100644
index 0000000..ee6f57d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java
@@ -0,0 +1,290 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+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.NamenodePriorityComparator;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.federation.resolver.PathLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
+import org.apache.hadoop.util.Time;
+
+/**
+ * In-memory cache/mock of a namenode and file resolver. Stores the most
+ * recently updated NN information for each nameservice and block pool. Also
+ * stores a virtual mount table for resolving global namespace paths to local NN
+ * paths.
+ */
+public class MockResolver
+    implements ActiveNamenodeResolver, FileSubclusterResolver {
+
+  private Map<String, List<? extends FederationNamenodeContext>> resolver =
+      new HashMap<String, List<? extends FederationNamenodeContext>>();
+  private Map<String, List<RemoteLocation>> locations =
+      new HashMap<String, List<RemoteLocation>>();
+  private Set<FederationNamespaceInfo> namespaces =
+      new HashSet<FederationNamespaceInfo>();
+  private String defaultNamespace = null;
+
+  public MockResolver(Configuration conf, StateStoreService store) {
+    this.cleanRegistrations();
+  }
+
+  public void addLocation(String mount, String nameservice, String location) {
+    RemoteLocation remoteLocation = new RemoteLocation(nameservice, location);
+    List<RemoteLocation> locationsList = locations.get(mount);
+    if (locationsList == null) {
+      locationsList = new LinkedList<RemoteLocation>();
+      locations.put(mount, locationsList);
+    }
+    if (!locationsList.contains(remoteLocation)) {
+      locationsList.add(remoteLocation);
+    }
+
+    if (this.defaultNamespace == null) {
+      this.defaultNamespace = nameservice;
+    }
+  }
+
+  public synchronized void cleanRegistrations() {
+    this.resolver =
+        new HashMap<String, List<? extends FederationNamenodeContext>>();
+    this.namespaces = new HashSet<FederationNamespaceInfo>();
+  }
+
+  @Override
+  public void updateActiveNamenode(
+      String ns, InetSocketAddress successfulAddress) {
+
+    String address = successfulAddress.getHostName() + ":" +
+        successfulAddress.getPort();
+    String key = ns;
+    if (key != null) {
+      // Update the active entry
+      @SuppressWarnings("unchecked")
+      List<FederationNamenodeContext> iterator =
+          (List<FederationNamenodeContext>) resolver.get(key);
+      for (FederationNamenodeContext namenode : iterator) {
+        if (namenode.getRpcAddress().equals(address)) {
+          MockNamenodeContext nn = (MockNamenodeContext) namenode;
+          nn.setState(FederationNamenodeServiceState.ACTIVE);
+          break;
+        }
+      }
+      Collections.sort(iterator, new NamenodePriorityComparator());
+    }
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext>
+      getNamenodesForNameserviceId(String nameserviceId) {
+    return resolver.get(nameserviceId);
+  }
+
+  @Override
+  public List<? extends FederationNamenodeContext> getNamenodesForBlockPoolId(
+      String blockPoolId) {
+    return resolver.get(blockPoolId);
+  }
+
+  private static class MockNamenodeContext
+      implements FederationNamenodeContext {
+    private String webAddress;
+    private String rpcAddress;
+    private String serviceAddress;
+    private String lifelineAddress;
+    private String namenodeId;
+    private String nameserviceId;
+    private FederationNamenodeServiceState state;
+    private long dateModified;
+
+    MockNamenodeContext(
+        String rpc, String service, String lifeline, String web,
+        String ns, String nn, FederationNamenodeServiceState state) {
+      this.rpcAddress = rpc;
+      this.serviceAddress = service;
+      this.lifelineAddress = lifeline;
+      this.webAddress = web;
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.state = state;
+      this.dateModified = Time.now();
+    }
+
+    public void setState(FederationNamenodeServiceState newState) {
+      this.state = newState;
+      this.dateModified = Time.now();
+    }
+
+    @Override
+    public String getRpcAddress() {
+      return rpcAddress;
+    }
+
+    @Override
+    public String getServiceAddress() {
+      return serviceAddress;
+    }
+
+    @Override
+    public String getLifelineAddress() {
+      return lifelineAddress;
+    }
+
+    @Override
+    public String getWebAddress() {
+      return webAddress;
+    }
+
+    @Override
+    public String getNamenodeKey() {
+      return nameserviceId + " " + namenodeId + " " + rpcAddress;
+    }
+
+    @Override
+    public String getNameserviceId() {
+      return nameserviceId;
+    }
+
+    @Override
+    public String getNamenodeId() {
+      return namenodeId;
+    }
+
+    @Override
+    public FederationNamenodeServiceState getState() {
+      return state;
+    }
+
+    @Override
+    public long getDateModified() {
+      return dateModified;
+    }
+  }
+
+  @Override
+  public synchronized boolean registerNamenode(NamenodeStatusReport report)
+      throws IOException {
+    MockNamenodeContext context = new MockNamenodeContext(
+        report.getRpcAddress(), report.getServiceAddress(),
+        report.getLifelineAddress(), report.getWebAddress(),
+        report.getNameserviceId(), report.getNamenodeId(), report.getState());
+
+    String nsId = report.getNameserviceId();
+    String bpId = report.getBlockPoolId();
+    String cId = report.getClusterId();
+    @SuppressWarnings("unchecked")
+    List<MockNamenodeContext> existingItems =
+        (List<MockNamenodeContext>) resolver.get(nsId);
+    if (existingItems == null) {
+      existingItems = new ArrayList<MockNamenodeContext>();
+      resolver.put(bpId, existingItems);
+      resolver.put(nsId, existingItems);
+    }
+    boolean added = false;
+    for (int i=0; i<existingItems.size() && !added; i++) {
+      MockNamenodeContext existing = existingItems.get(i);
+      if (existing.getNamenodeKey().equals(context.getNamenodeKey())) {
+        existingItems.set(i, context);
+        added = true;
+      }
+    }
+    if (!added) {
+      existingItems.add(context);
+    }
+    Collections.sort(existingItems, new NamenodePriorityComparator());
+
+    FederationNamespaceInfo info = new FederationNamespaceInfo(bpId, cId, nsId);
+    namespaces.add(info);
+    return true;
+  }
+
+  @Override
+  public Set<FederationNamespaceInfo> getNamespaces() throws IOException {
+    return this.namespaces;
+  }
+
+  @Override
+  public PathLocation getDestinationForPath(String path) throws IOException {
+    String finalPath = null;
+    String nameservice = null;
+    Set<String> namespaceSet = new HashSet<String>();
+    LinkedList<RemoteLocation> remoteLocations =
+        new LinkedList<RemoteLocation>();
+    for(String key : this.locations.keySet()) {
+      if(path.startsWith(key)) {
+        for (RemoteLocation location : this.locations.get(key)) {
+          finalPath = location.getDest() + path.substring(key.length());
+          nameservice = location.getNameserviceId();
+          RemoteLocation remoteLocation =
+              new RemoteLocation(nameservice, finalPath);
+          remoteLocations.add(remoteLocation);
+          namespaceSet.add(nameservice);
+        }
+        break;
+      }
+    }
+    if (remoteLocations.isEmpty()) {
+      // Path isn't supported, mimic resolver behavior.
+      return null;
+    }
+    return new PathLocation(path, remoteLocations, namespaceSet);
+  }
+
+  @Override
+  public List<String> getMountPoints(String path) throws IOException {
+    List<String> mounts = new ArrayList<String>();
+    if (path.equals("/")) {
+      // Mounts only supported under root level
+      for (String mount : this.locations.keySet()) {
+        if (mount.length() > 1) {
+          // Remove leading slash, this is the behavior of the mount tree,
+          // return only names.
+          mounts.add(mount.replace("/", ""));
+        }
+      }
+    }
+    return mounts;
+  }
+
+  @Override
+  public void setRouterId(String router) {
+  }
+
+  @Override
+  public String getDefaultNamespace() {
+    return defaultNamespace;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.java
new file mode 100644
index 0000000..16d624c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterConfigBuilder.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
+ *
+ *     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;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Constructs a router configuration with individual features enabled/disabled.
+ */
+public class RouterConfigBuilder {
+
+  private Configuration conf;
+
+  public RouterConfigBuilder(Configuration configuration) {
+    this.conf = configuration;
+  }
+
+  public RouterConfigBuilder() {
+    this.conf = new Configuration();
+  }
+
+  public Configuration build() {
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
new file mode 100644
index 0000000..55d04ad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -0,0 +1,767 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
+import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
+import org.apache.hadoop.hdfs.server.federation.router.Router;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service.STATE;
+
+/**
+ * Test utility to mimic a federated HDFS cluster with a router.
+ */
+public class RouterDFSCluster {
+  /**
+   * Router context.
+   */
+  public class RouterContext {
+    private Router router;
+    private FileContext fileContext;
+    private String nameserviceId;
+    private String namenodeId;
+    private int rpcPort;
+    private DFSClient client;
+    private Configuration conf;
+    private URI fileSystemUri;
+
+    public RouterContext(Configuration conf, String ns, String nn)
+        throws URISyntaxException {
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.conf = conf;
+      router = new Router();
+      router.init(conf);
+    }
+
+    public Router getRouter() {
+      return this.router;
+    }
+
+    public String getNameserviceId() {
+      return this.nameserviceId;
+    }
+
+    public String getNamenodeId() {
+      return this.namenodeId;
+    }
+
+    public int getRpcPort() {
+      return this.rpcPort;
+    }
+
+    public FileContext getFileContext() {
+      return this.fileContext;
+    }
+
+    public void initRouter() throws URISyntaxException {
+    }
+
+    public DistributedFileSystem getFileSystem() throws IOException {
+      DistributedFileSystem fs =
+          (DistributedFileSystem) DistributedFileSystem.get(conf);
+      return fs;
+    }
+
+    public DFSClient getClient(UserGroupInformation user)
+        throws IOException, URISyntaxException, InterruptedException {
+
+      LOG.info("Connecting to router at " + fileSystemUri);
+      return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
+        @Override
+        public DFSClient run() throws IOException {
+          return new DFSClient(fileSystemUri, conf);
+        }
+      });
+    }
+
+    public DFSClient getClient() throws IOException, URISyntaxException {
+
+      if (client == null) {
+        LOG.info("Connecting to router at " + fileSystemUri);
+        client = new DFSClient(fileSystemUri, conf);
+      }
+      return client;
+    }
+  }
+
+  /**
+   * Namenode context.
+   */
+  public class NamenodeContext {
+    private NameNode namenode;
+    private String nameserviceId;
+    private String namenodeId;
+    private FileContext fileContext;
+    private int rpcPort;
+    private int servicePort;
+    private int lifelinePort;
+    private int httpPort;
+    private URI fileSystemUri;
+    private int index;
+    private Configuration conf;
+    private DFSClient client;
+
+    public NamenodeContext(Configuration conf, String ns, String nn,
+        int index) {
+      this.conf = conf;
+      this.namenodeId = nn;
+      this.nameserviceId = ns;
+      this.index = index;
+    }
+
+    public NameNode getNamenode() {
+      return this.namenode;
+    }
+
+    public String getNameserviceId() {
+      return this.nameserviceId;
+    }
+
+    public String getNamenodeId() {
+      return this.namenodeId;
+    }
+
+    public FileContext getFileContext() {
+      return this.fileContext;
+    }
+
+    public void setNamenode(NameNode n) throws URISyntaxException {
+      namenode = n;
+
+      // Store the bound ports and override the default FS with the local NN's
+      // RPC
+      rpcPort = n.getNameNodeAddress().getPort();
+      servicePort = n.getServiceRpcAddress().getPort();
+      lifelinePort = n.getServiceRpcAddress().getPort();
+      httpPort = n.getHttpAddress().getPort();
+      fileSystemUri = new URI("hdfs://" + namenode.getHostAndPort());
+      DistributedFileSystem.setDefaultUri(conf, fileSystemUri);
+
+      try {
+        this.fileContext = FileContext.getFileContext(conf);
+      } catch (UnsupportedFileSystemException e) {
+        this.fileContext = null;
+      }
+    }
+
+    public String getRpcAddress() {
+      return namenode.getNameNodeAddress().getHostName() + ":" + rpcPort;
+    }
+
+    public String getServiceAddress() {
+      return namenode.getServiceRpcAddress().getHostName() + ":" + servicePort;
+    }
+
+    public String getLifelineAddress() {
+      return namenode.getServiceRpcAddress().getHostName() + ":" + lifelinePort;
+    }
+
+    public String getHttpAddress() {
+      return namenode.getHttpAddress().getHostName() + ":" + httpPort;
+    }
+
+    public DistributedFileSystem getFileSystem() throws IOException {
+      DistributedFileSystem fs =
+          (DistributedFileSystem) DistributedFileSystem.get(conf);
+      return fs;
+    }
+
+    public void resetClient() {
+      client = null;
+    }
+
+    public DFSClient getClient(UserGroupInformation user)
+        throws IOException, URISyntaxException, InterruptedException {
+
+      LOG.info("Connecting to namenode at " + fileSystemUri);
+      return user.doAs(new PrivilegedExceptionAction<DFSClient>() {
+        @Override
+        public DFSClient run() throws IOException {
+          return new DFSClient(fileSystemUri, conf);
+        }
+      });
+    }
+
+    public DFSClient getClient() throws IOException, URISyntaxException {
+      if (client == null) {
+        LOG.info("Connecting to namenode at " + fileSystemUri);
+        client = new DFSClient(fileSystemUri, conf);
+      }
+      return client;
+    }
+
+    public String getConfSuffix() {
+      String suffix = nameserviceId;
+      if (highAvailability) {
+        suffix += "." + namenodeId;
+      }
+      return suffix;
+    }
+  }
+
+  public static final String NAMENODE1 = "nn0";
+  public static final String NAMENODE2 = "nn1";
+  public static final String NAMENODE3 = "nn2";
+  public static final String TEST_STRING = "teststring";
+  public static final String TEST_DIR = "testdir";
+  public static final String TEST_FILE = "testfile";
+
+  private List<String> nameservices;
+  private List<RouterContext> routers;
+  private List<NamenodeContext> namenodes;
+  private static final Log LOG = LogFactory.getLog(RouterDFSCluster.class);
+  private MiniDFSCluster cluster;
+  private boolean highAvailability;
+
+  protected static final int DEFAULT_HEARTBEAT_INTERVAL = 5;
+  protected static final int DEFAULT_CACHE_INTERVAL_SEC = 5;
+  private Configuration routerOverrides;
+  private Configuration namenodeOverrides;
+
+  private static final String NAMENODES = NAMENODE1 + "," + NAMENODE2;
+
+  public RouterDFSCluster(boolean ha, int numNameservices) {
+    this(ha, numNameservices, 2);
+  }
+
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
+    this.highAvailability = ha;
+    configureNameservices(numNameservices, numNamenodes);
+  }
+
+  public void addRouterOverrides(Configuration conf) {
+    if (this.routerOverrides == null) {
+      this.routerOverrides = conf;
+    } else {
+      this.routerOverrides.addResource(conf);
+    }
+  }
+
+  public void addNamenodeOverrides(Configuration conf) {
+    if (this.namenodeOverrides == null) {
+      this.namenodeOverrides = conf;
+    } else {
+      this.namenodeOverrides.addResource(conf);
+    }
+  }
+
+  public Configuration generateNamenodeConfiguration(
+      String defaultNameserviceId) {
+    Configuration c = new HdfsConfiguration();
+
+    c.set(DFSConfigKeys.DFS_NAMESERVICES, getNameservicesKey());
+    c.set("fs.defaultFS", "hdfs://" + defaultNameserviceId);
+
+    for (String ns : nameservices) {
+      if (highAvailability) {
+        c.set(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns, NAMENODES);
+      }
+
+      for (NamenodeContext context : getNamenodes(ns)) {
+        String suffix = context.getConfSuffix();
+
+        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + suffix,
+            "127.0.0.1:" + context.rpcPort);
+        c.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + suffix,
+            "127.0.0.1:" + context.httpPort);
+        c.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + suffix,
+            "0.0.0.0");
+      }
+    }
+
+    if (namenodeOverrides != null) {
+      c.addResource(namenodeOverrides);
+    }
+    return c;
+  }
+
+  public Configuration generateClientConfiguration() {
+    Configuration conf = new HdfsConfiguration();
+    conf.addResource(generateNamenodeConfiguration(getNameservices().get(0)));
+    return conf;
+  }
+
+  public Configuration generateRouterConfiguration(String localNameserviceId,
+      String localNamenodeId) throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.addResource(generateNamenodeConfiguration(localNameserviceId));
+
+    // Use mock resolver classes
+    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+
+    // Set the nameservice ID for the default NN monitor
+    conf.set(DFSConfigKeys.DFS_NAMESERVICE_ID, localNameserviceId);
+
+    if (localNamenodeId != null) {
+      conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, localNamenodeId);
+    }
+
+    StringBuilder routerBuilder = new StringBuilder();
+    for (String ns : nameservices) {
+      for (NamenodeContext context : getNamenodes(ns)) {
+        String suffix = context.getConfSuffix();
+
+        if (routerBuilder.length() != 0) {
+          routerBuilder.append(",");
+        }
+        routerBuilder.append(suffix);
+      }
+    }
+
+    return conf;
+  }
+
+  public void configureNameservices(int numNameservices, int numNamenodes) {
+    nameservices = new ArrayList<String>();
+    for (int i = 0; i < numNameservices; i++) {
+      nameservices.add("ns" + i);
+    }
+    namenodes = new ArrayList<NamenodeContext>();
+    int index = 0;
+    for (String ns : nameservices) {
+      Configuration nnConf = generateNamenodeConfiguration(ns);
+      if (highAvailability) {
+        NamenodeContext context =
+            new NamenodeContext(nnConf, ns, NAMENODE1, index);
+        namenodes.add(context);
+        index++;
+
+        if (numNamenodes > 1) {
+          context = new NamenodeContext(nnConf, ns, NAMENODE2, index + 1);
+          namenodes.add(context);
+          index++;
+        }
+
+        if (numNamenodes > 2) {
+          context = new NamenodeContext(nnConf, ns, NAMENODE3, index + 1);
+          namenodes.add(context);
+          index++;
+        }
+
+      } else {
+        NamenodeContext context = new NamenodeContext(nnConf, ns, null, index);
+        namenodes.add(context);
+        index++;
+      }
+    }
+  }
+
+  public String getNameservicesKey() {
+    StringBuilder ns = new StringBuilder();
+    for (int i = 0; i < nameservices.size(); i++) {
+      if (i > 0) {
+        ns.append(",");
+      }
+      ns.append(nameservices.get(i));
+    }
+    return ns.toString();
+  }
+
+  public String getRandomNameservice() {
+    Random r = new Random();
+    return nameservices.get(r.nextInt(nameservices.size()));
+  }
+
+  public List<String> getNameservices() {
+    return nameservices;
+  }
+
+  public List<NamenodeContext> getNamenodes(String nameservice) {
+    ArrayList<NamenodeContext> nns = new ArrayList<NamenodeContext>();
+    for (NamenodeContext c : namenodes) {
+      if (c.nameserviceId.equals(nameservice)) {
+        nns.add(c);
+      }
+    }
+    return nns;
+  }
+
+  public NamenodeContext getRandomNamenode() {
+    Random rand = new Random();
+    return namenodes.get(rand.nextInt(namenodes.size()));
+  }
+
+  public List<NamenodeContext> getNamenodes() {
+    return namenodes;
+  }
+
+  public boolean isHighAvailability() {
+    return highAvailability;
+  }
+
+  public NamenodeContext getNamenode(String nameservice,
+      String namenode) {
+    for (NamenodeContext c : namenodes) {
+      if (c.nameserviceId.equals(nameservice)) {
+        if (namenode == null || c.namenodeId == null || namenode.isEmpty()
+            || c.namenodeId.isEmpty()) {
+          return c;
+        } else if (c.namenodeId.equals(namenode)) {
+          return c;
+        }
+      }
+    }
+    return null;
+  }
+
+  public List<RouterContext> getRouters(String nameservice) {
+    ArrayList<RouterContext> nns = new ArrayList<RouterContext>();
+    for (RouterContext c : routers) {
+      if (c.nameserviceId.equals(nameservice)) {
+        nns.add(c);
+      }
+    }
+    return nns;
+  }
+
+  public RouterContext getRouterContext(String nameservice,
+      String namenode) {
+    for (RouterContext c : routers) {
+      if (namenode == null) {
+        return c;
+      }
+      if (c.namenodeId.equals(namenode)
+          && c.nameserviceId.equals(nameservice)) {
+        return c;
+      }
+    }
+    return null;
+  }
+
+  public RouterContext getRandomRouter() {
+    Random rand = new Random();
+    return routers.get(rand.nextInt(routers.size()));
+  }
+
+  public List<RouterContext> getRouters() {
+    return routers;
+  }
+
+  public RouterContext buildRouter(String nameservice, String namenode)
+      throws URISyntaxException, IOException {
+    Configuration config = generateRouterConfiguration(nameservice, namenode);
+    RouterContext rc = new RouterContext(config, nameservice, namenode);
+    return rc;
+  }
+
+  public void startCluster() {
+    startCluster(null);
+  }
+
+  public void startCluster(Configuration overrideConf) {
+    try {
+      MiniDFSNNTopology topology = new MiniDFSNNTopology();
+      for (String ns : nameservices) {
+
+        NSConf conf = new MiniDFSNNTopology.NSConf(ns);
+        if (highAvailability) {
+          for(int i = 0; i < namenodes.size()/nameservices.size(); i++) {
+            NNConf nnConf = new MiniDFSNNTopology.NNConf("nn" + i);
+            conf.addNN(nnConf);
+          }
+        } else {
+          NNConf nnConf = new MiniDFSNNTopology.NNConf(null);
+          conf.addNN(nnConf);
+        }
+        topology.addNameservice(conf);
+      }
+      topology.setFederation(true);
+
+      // Start mini DFS cluster
+      Configuration nnConf = generateNamenodeConfiguration(nameservices.get(0));
+      if (overrideConf != null) {
+        nnConf.addResource(overrideConf);
+      }
+      cluster = new MiniDFSCluster.Builder(nnConf).nnTopology(topology).build();
+      cluster.waitActive();
+
+      // Store NN pointers
+      for (int i = 0; i < namenodes.size(); i++) {
+        NameNode nn = cluster.getNameNode(i);
+        namenodes.get(i).setNamenode(nn);
+      }
+
+    } catch (Exception e) {
+      LOG.error("Cannot start Router DFS cluster: " + e.getMessage(), e);
+      cluster.shutdown();
+    }
+  }
+
+  public void startRouters()
+      throws InterruptedException, URISyntaxException, IOException {
+    // Create routers
+    routers = new ArrayList<RouterContext>();
+    for (String ns : nameservices) {
+
+      for (NamenodeContext context : getNamenodes(ns)) {
+        routers.add(buildRouter(ns, context.namenodeId));
+      }
+    }
+
+    // Start all routers
+    for (RouterContext router : routers) {
+      router.router.start();
+    }
+    // Wait until all routers are active and record their ports
+    for (RouterContext router : routers) {
+      waitActive(router);
+      router.initRouter();
+    }
+  }
+
+  public void waitActive(NamenodeContext nn) throws IOException {
+    cluster.waitActive(nn.index);
+  }
+
+  public void waitActive(RouterContext router)
+      throws InterruptedException {
+    for (int loopCount = 0; loopCount < 20; loopCount++) {
+      // Validate connection of routers to NNs
+      if (router.router.getServiceState() == STATE.STARTED) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    assertFalse(
+        "Timeout waiting for " + router.router.toString() + " to activate.",
+        true);
+  }
+
+
+  public void registerNamenodes() throws IOException {
+    for (RouterContext r : routers) {
+      ActiveNamenodeResolver resolver = r.router.getNamenodeResolver();
+      for (NamenodeContext nn : namenodes) {
+        // Generate a report
+        NamenodeStatusReport report = new NamenodeStatusReport(nn.nameserviceId,
+            nn.namenodeId, nn.getRpcAddress(), nn.getServiceAddress(),
+            nn.getLifelineAddress(), nn.getHttpAddress());
+        report.setNamespaceInfo(nn.namenode.getNamesystem().getFSImage()
+            .getStorage().getNamespaceInfo());
+
+        // Determine HA state from nn public state string
+        String nnState = nn.namenode.getState();
+        HAServiceState haState = HAServiceState.ACTIVE;
+        for (HAServiceState state : HAServiceState.values()) {
+          if (nnState.equalsIgnoreCase(state.name())) {
+            haState = state;
+            break;
+          }
+        }
+        report.setHAServiceState(haState);
+
+        // Register with the resolver
+        resolver.registerNamenode(report);
+      }
+    }
+  }
+
+  public void waitNamenodeRegistration()
+      throws InterruptedException, IllegalStateException, IOException {
+    for (RouterContext r : routers) {
+      for (NamenodeContext nn : namenodes) {
+        FederationTestUtils.waitNamenodeRegistered(
+            r.router.getNamenodeResolver(), nn.nameserviceId, nn.namenodeId,
+            null);
+      }
+    }
+  }
+
+  public void waitRouterRegistrationQuorum(RouterContext router,
+      FederationNamenodeServiceState state, String nameservice, String namenode)
+          throws InterruptedException, IOException {
+    LOG.info("Waiting for NN - " + nameservice + ":" + namenode
+        + " to transition to state - " + state);
+    FederationTestUtils.waitNamenodeRegistered(
+        router.router.getNamenodeResolver(), nameservice, namenode, state);
+  }
+
+  public String getFederatedPathForNameservice(String ns) {
+    return "/" + ns;
+  }
+
+  public String getNamenodePathForNameservice(String ns) {
+    return "/target-" + ns;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/ns0/testdir which maps to ns0->/target-ns0/testdir
+   *         </ul>
+   */
+  public String getFederatedTestDirectoryForNameservice(String ns) {
+    return getFederatedPathForNameservice(ns) + "/" + TEST_DIR;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/target-ns0/testdir
+   *         </ul>
+   */
+  public String getNamenodeTestDirectoryForNameservice(String ns) {
+    return getNamenodePathForNameservice(ns) + "/" + TEST_DIR;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/ns0/testfile which maps to ns0->/target-ns0/testfile
+   *         </ul>
+   */
+  public String getFederatedTestFileForNameservice(String ns) {
+    return getFederatedPathForNameservice(ns) + "/" + TEST_FILE;
+  }
+
+  /**
+   * @return example:
+   *         <ul>
+   *         <li>/target-ns0/testfile
+   *         </ul>
+   */
+  public String getNamenodeTestFileForNameservice(String ns) {
+    return getNamenodePathForNameservice(ns) + "/" + TEST_FILE;
+  }
+
+  public void shutdown() {
+    cluster.shutdown();
+    if (routers != null) {
+      for (RouterContext context : routers) {
+        stopRouter(context);
+      }
+    }
+  }
+
+  public void stopRouter(RouterContext router) {
+    try {
+
+      router.router.shutDown();
+
+      int loopCount = 0;
+      while (router.router.getServiceState() != STATE.STOPPED) {
+        loopCount++;
+        Thread.sleep(1000);
+        if (loopCount > 20) {
+          LOG.error("Unable to shutdown router - " + router.rpcPort);
+          break;
+        }
+      }
+    } catch (InterruptedException e) {
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // Namespace Test Fixtures
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * Creates test directories via the namenode.
+   * 1) /target-ns0/testfile
+   * 2) /target-ns1/testfile
+   * @throws IOException
+   */
+  public void createTestDirectoriesNamenode() throws IOException {
+    // Add a test dir to each NS and verify
+    for (String ns : getNameservices()) {
+      NamenodeContext context = getNamenode(ns, null);
+      if (!createTestDirectoriesNamenode(context)) {
+        throw new IOException("Unable to create test directory for ns - " + ns);
+      }
+    }
+  }
+
+  public boolean createTestDirectoriesNamenode(NamenodeContext nn)
+      throws IOException {
+    return FederationTestUtils.addDirectory(nn.getFileSystem(),
+        getNamenodeTestDirectoryForNameservice(nn.nameserviceId));
+  }
+
+  public void deleteAllFiles() throws IOException {
+    // Delete all files via the NNs and verify
+    for (NamenodeContext context : getNamenodes()) {
+      FileStatus[] status = context.getFileSystem().listStatus(new Path("/"));
+      for(int i = 0; i <status.length; i++) {
+        Path p = status[i].getPath();
+        context.getFileSystem().delete(p, true);
+      }
+      status = context.getFileSystem().listStatus(new Path("/"));
+      assertEquals(status.length, 0);
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // MockRouterResolver Test Fixtures
+  /////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * <ul>
+   * <li>/ -> [ns0->/].
+   * <li>/nso -> ns0->/target-ns0.
+   * <li>/ns1 -> ns1->/target-ns1.
+   * </ul>
+   */
+  public void installMockLocations() {
+    for (RouterContext r : routers) {
+      MockResolver resolver =
+          (MockResolver) r.router.getSubclusterResolver();
+      // create table entries
+      for (String ns : nameservices) {
+        // Direct path
+        resolver.addLocation(getFederatedPathForNameservice(ns), ns,
+            getNamenodePathForNameservice(ns));
+      }
+
+      // Root path goes to both NS1
+      resolver.addLocation("/", nameservices.get(0), "/");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/deb0a671/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
new file mode 100644
index 0000000..8c720c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java
@@ -0,0 +1,96 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.federation.MockResolver;
+import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
+import org.apache.hadoop.service.Service.STATE;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * The the safe mode for the {@link Router} controlled by
+ * {@link SafeModeTimer}.
+ */
+public class TestRouter {
+
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void create() throws IOException {
+    // Basic configuration without the state store
+    conf = new Configuration();
+    // Mock resolver classes
+    conf.set(DFSConfigKeys.FEDERATION_NAMENODE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+    conf.set(DFSConfigKeys.FEDERATION_FILE_RESOLVER_CLIENT_CLASS,
+        MockResolver.class.getCanonicalName());
+
+    // Simulate a co-located NN
+    conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns0");
+    conf.set("fs.defaultFS", "hdfs://" + "ns0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + "ns0",
+            "127.0.0.1:0" + 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY + "." + "ns0",
+            "127.0.0.1:" + 0);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY + "." + "ns0",
+            "0.0.0.0");
+  }
+
+  @AfterClass
+  public static void destroy() {
+  }
+
+  @Before
+  public void setup() throws IOException, URISyntaxException {
+  }
+
+  @After
+  public void cleanup() {
+  }
+
+  private static void testRouterStartup(Configuration routerConfig)
+      throws InterruptedException, IOException {
+    Router router = new Router();
+    assertEquals(STATE.NOTINITED, router.getServiceState());
+    router.init(routerConfig);
+    assertEquals(STATE.INITED, router.getServiceState());
+    router.start();
+    assertEquals(STATE.STARTED, router.getServiceState());
+    router.stop();
+    assertEquals(STATE.STOPPED, router.getServiceState());
+    router.close();
+  }
+
+  @Test
+  public void testRouterService() throws InterruptedException, IOException {
+
+    // Run with all services
+    testRouterStartup((new RouterConfigBuilder(conf)).build());
+  }
+}


---------------------------------------------------------------------
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: HDFS-11896. Non-dfsUsed will be doubled on dead node re-registration. Contributed by Brahma Reddy Battula.

Posted by in...@apache.org.
HDFS-11896. Non-dfsUsed will be doubled on dead node re-registration. Contributed by Brahma Reddy Battula.

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

Branch: refs/heads/HDFS-10467
Commit: c4a85c694fae3f814ab4e7f3c172da1df0e0e353
Parents: 11ece0b
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Thu Jul 27 12:02:57 2017 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Thu Jul 27 12:02:57 2017 -0700

----------------------------------------------------------------------
 .../blockmanagement/DatanodeDescriptor.java     | 19 ++++---
 .../hadoop/hdfs/server/datanode/DataNode.java   |  2 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  | 53 ++++++++++++++++++++
 3 files changed, 65 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4a85c69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 57348a3..2bd4a20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -331,11 +331,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   public void resetBlocks() {
-    setCapacity(0);
-    setRemaining(0);
-    setBlockPoolUsed(0);
-    setDfsUsed(0);
-    setXceiverCount(0);
+    updateStorageStats(this.getStorageReports(), 0L, 0L, 0, 0, null);
     this.invalidateBlocks.clear();
     this.volumeFailures = 0;
     // pendingCached, cached, and pendingUncached are protected by the
@@ -384,6 +380,16 @@ public class DatanodeDescriptor extends DatanodeInfo {
   public void updateHeartbeatState(StorageReport[] reports, long cacheCapacity,
       long cacheUsed, int xceiverCount, int volFailures,
       VolumeFailureSummary volumeFailureSummary) {
+    updateStorageStats(reports, cacheCapacity, cacheUsed, xceiverCount,
+        volFailures, volumeFailureSummary);
+    setLastUpdate(Time.now());
+    setLastUpdateMonotonic(Time.monotonicNow());
+    rollBlocksScheduled(getLastUpdateMonotonic());
+  }
+
+  private void updateStorageStats(StorageReport[] reports, long cacheCapacity,
+      long cacheUsed, int xceiverCount, int volFailures,
+      VolumeFailureSummary volumeFailureSummary) {
     long totalCapacity = 0;
     long totalRemaining = 0;
     long totalBlockPoolUsed = 0;
@@ -434,8 +440,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setCacheCapacity(cacheCapacity);
     setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
-    setLastUpdate(Time.now());
-    setLastUpdateMonotonic(Time.monotonicNow());
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
@@ -451,7 +455,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
       totalDfsUsed += report.getDfsUsed();
       totalNonDfsUsed += report.getNonDfsUsed();
     }
-    rollBlocksScheduled(getLastUpdateMonotonic());
 
     // Update total metrics for the node.
     setCapacity(totalCapacity);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4a85c69/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 35fbb9c..2730393 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1319,7 +1319,7 @@ public class DataNode extends ReconfigurableBase
 
   // used only for testing
   @VisibleForTesting
-  void setHeartbeatsDisabledForTests(
+  public void setHeartbeatsDisabledForTests(
       boolean heartbeatsDisabledForTests) {
     this.heartbeatsDisabledForTests = heartbeatsDisabledForTests;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4a85c69/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 6df8fcf..74be90c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import com.google.common.base.Supplier;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -36,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.protocol.BlockType;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -52,6 +55,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
 
@@ -178,4 +182,53 @@ public class TestDeadDatanode {
           .getDatanodeDescriptor().equals(clientNode));
     }
   }
+
+  @Test
+  public void testNonDFSUsedONDeadNodeReReg() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
+        6 * 1000);
+    long CAPACITY = 5000L;
+    long[] capacities = new long[] { 4 * CAPACITY, 4 * CAPACITY };
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+          .simulatedCapacities(capacities).build();
+      long initialCapacity = cluster.getNamesystem(0).getCapacityTotal();
+      assertTrue(initialCapacity > 0);
+      DataNode dn1 = cluster.getDataNodes().get(0);
+      DataNode dn2 = cluster.getDataNodes().get(1);
+      final DatanodeDescriptor dn2Desc = cluster.getNamesystem(0)
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn2.getDatanodeId());
+      dn1.setHeartbeatsDisabledForTests(true);
+      cluster.setDataNodeDead(dn1.getDatanodeId());
+      assertEquals("Capacity shouldn't include DeadNode", dn2Desc.getCapacity(),
+          cluster.getNamesystem(0).getCapacityTotal());
+      assertEquals("NonDFS-used shouldn't include DeadNode",
+          dn2Desc.getNonDfsUsed(),
+          cluster.getNamesystem(0).getNonDfsUsedSpace());
+      // Wait for re-registration and heartbeat
+      dn1.setHeartbeatsDisabledForTests(false);
+      final DatanodeDescriptor dn1Desc = cluster.getNamesystem(0)
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn1.getDatanodeId());
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+
+        @Override public Boolean get() {
+          return dn1Desc.isAlive() && dn1Desc.isHeartbeatedSinceRegistration();
+        }
+      }, 100, 5000);
+      assertEquals("Capacity should be 0 after all DNs dead", initialCapacity,
+          cluster.getNamesystem(0).getCapacityTotal());
+      long nonDfsAfterReg = cluster.getNamesystem(0).getNonDfsUsedSpace();
+      assertEquals("NonDFS should include actual DN NonDFSUsed",
+          dn1Desc.getNonDfsUsed() + dn2Desc.getNonDfsUsed(), nonDfsAfterReg);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


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