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 wa...@apache.org on 2015/09/04 23:42:06 UTC

[01/50] [abbrv] hadoop git commit: HDFS-8155. Support OAuth2 in WebHDFS.

Repository: hadoop
Updated Branches:
  refs/heads/YARN-1197 f35a94517 -> f86eae17d (forced update)


HDFS-8155. Support OAuth2 in WebHDFS.


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

Branch: refs/heads/YARN-1197
Commit: 837fb75e8e03b2f016bcea2f4605106a5022491c
Parents: 6ab2d19
Author: Jakob Homan <jg...@gmail.com>
Authored: Sat Aug 29 18:37:05 2015 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Sat Aug 29 18:37:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |   5 +
 .../hdfs/client/HdfsClientConfigKeys.java       |   8 +
 .../hadoop/hdfs/web/URLConnectionFactory.java   |  30 ++-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  15 +-
 .../hdfs/web/oauth2/AccessTokenProvider.java    |  66 ++++++
 .../hdfs/web/oauth2/AccessTokenTimer.java       | 103 +++++++++
 .../ConfCredentialBasedAccessTokenProvider.java |  62 ++++++
 ...onfRefreshTokenBasedAccessTokenProvider.java | 146 +++++++++++++
 .../CredentialBasedAccessTokenProvider.java     | 135 ++++++++++++
 .../oauth2/OAuth2ConnectionConfigurator.java    |  79 +++++++
 .../hadoop/hdfs/web/oauth2/OAuth2Constants.java |  46 ++++
 .../apache/hadoop/hdfs/web/oauth2/Utils.java    |  63 ++++++
 .../hadoop/hdfs/web/oauth2/package-info.java    |  26 +++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   6 +
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |  25 +++
 .../hadoop/hdfs/web/TestWebHDFSOAuth2.java      | 216 +++++++++++++++++++
 .../hdfs/web/oauth2/TestAccessTokenTimer.java   |  63 ++++++
 ...ClientCredentialTimeBasedTokenRefresher.java | 138 ++++++++++++
 ...TestRefreshTokenTimeBasedTokenRefresher.java | 138 ++++++++++++
 20 files changed, 1369 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index aeaa980..68bd289 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -32,6 +32,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <dependencies>
     <dependency>
+      <groupId>com.squareup.okhttp</groupId>
+      <artifactId>okhttp</artifactId>
+      <version>2.4.0</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 7b1e438..96bc8d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -36,6 +36,14 @@ public interface HdfsClientConfigKeys {
   String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       "^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
 
+  String DFS_WEBHDFS_OAUTH_ENABLED_KEY = "dfs.webhdfs.oauth2.enabled";
+  boolean DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT = false;
+
+  String OAUTH_CLIENT_ID_KEY = "dfs.webhdfs.oauth2.client.id";
+  String OAUTH_REFRESH_URL_KEY = "dfs.webhdfs.oauth2.refresh.url";
+
+  String ACCESS_TOKEN_PROVIDER_KEY = "dfs.webhdfs.oauth2.access.token.provider";
+
   String PREFIX = "dfs.client.";
   String  DFS_NAMESERVICES = "dfs.nameservices";
   int     DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index a5e02f2..4c23241 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -31,6 +31,7 @@ import javax.net.ssl.SSLSocketFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
@@ -77,15 +78,42 @@ public class URLConnectionFactory {
    * try to load SSL certificates when it is specified.
    */
   public static URLConnectionFactory newDefaultURLConnectionFactory(Configuration conf) {
+    ConnectionConfigurator conn = getSSLConnectionConfiguration(conf);
+
+    return new URLConnectionFactory(conn);
+  }
+
+  private static ConnectionConfigurator
+      getSSLConnectionConfiguration(Configuration conf) {
     ConnectionConfigurator conn = null;
     try {
       conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
     } catch (Exception e) {
       LOG.debug(
-          "Cannot load customized ssl related configuration. Fallback to system-generic settings.",
+          "Cannot load customized ssl related configuration. Fallback to" +
+              " system-generic settings.",
           e);
       conn = DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
     }
+
+    return conn;
+  }
+
+  /**
+   * Construct a new URLConnectionFactory that supports OAut-based connections.
+   * It will also try to load the SSL configuration when they are specified.
+   */
+  public static URLConnectionFactory
+      newOAuth2URLConnectionFactory(Configuration conf) throws IOException {
+    ConnectionConfigurator conn = null;
+    try {
+      ConnectionConfigurator sslConnConfigurator
+          = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
+
+      conn = new OAuth2ConnectionConfigurator(conf, sslConnConfigurator);
+    } catch (Exception e) {
+      throw new IOException("Unable to load OAuth2 connection factory.", e);
+    }
     return new URLConnectionFactory(conn);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index ee5238a..a75e78f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -149,8 +149,19 @@ public class WebHdfsFileSystem extends FileSystem
         HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
         HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
 
-    connectionFactory = URLConnectionFactory
-        .newDefaultURLConnectionFactory(conf);
+    boolean isOAuth = conf.getBoolean(
+        HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY,
+        HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT);
+
+    if(isOAuth) {
+      LOG.info("Enabling OAuth2 in WebHDFS");
+      connectionFactory = URLConnectionFactory
+          .newOAuth2URLConnectionFactory(conf);
+    } else {
+      LOG.info("Not enabling OAuth2 in WebHDFS");
+      connectionFactory = URLConnectionFactory
+          .newDefaultURLConnectionFactory(conf);
+    }
 
 
     ugi = UserGroupInformation.getCurrentUser();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.java
new file mode 100644
index 0000000..99e153d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenProvider.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.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+/**
+ * Provide an OAuth2 access token to be used to authenticate http calls in
+ * WebHDFS.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class AccessTokenProvider implements Configurable {
+  private Configuration conf;
+
+  /**
+   * Obtain the access token that should be added to http connection's header.
+   * Will be called for each connection, so implementations should be
+   * performant. Implementations are responsible for any refreshing of
+   * the token.
+   * 
+   * @return Access token to be added to connection header.
+   */
+  abstract String getAccessToken() throws IOException;
+
+  /**
+   * Return the conf.
+   *
+   * @return the conf.
+   */
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Set the conf.
+   *
+   * @param configuration  New configuration.
+   */
+  @Override
+  public void setConf(Configuration configuration) {
+    this.conf = configuration;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java
new file mode 100644
index 0000000..aa05dd6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AccessTokenTimer.java
@@ -0,0 +1,103 @@
+/*
+ * 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.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.Timer;
+
+/**
+ * Access tokens generally expire.  This timer helps keep track of that.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AccessTokenTimer {
+  public static final long EXPIRE_BUFFER_MS = 30 * 1000L;
+
+  private final Timer timer;
+
+  /**
+   * When the current access token will expire in milliseconds since
+   * epoch.
+   */
+  private long nextRefreshMSSinceEpoch;
+
+  public AccessTokenTimer() {
+    this(new Timer());
+  }
+
+  /**
+   * 
+   * @param timer Timer instance for unit testing
+   */
+  public AccessTokenTimer(Timer timer) {
+    this.timer = timer;
+    this.nextRefreshMSSinceEpoch = 0;
+  }
+
+  /** 
+   * Set when the access token will expire as reported by the oauth server,
+   * ie in seconds from now.
+   * @param expiresIn Access time expiration as reported by OAuth server
+   */
+  public void setExpiresIn(String expiresIn) {
+    this.nextRefreshMSSinceEpoch = convertExpiresIn(timer, expiresIn);
+  }
+
+  /**
+   * Set when the access token will expire in milliseconds from epoch,
+   * as required by the WebHDFS configuration.  This is a bit hacky and lame.
+   * 
+   * @param expiresInMSSinceEpoch Access time expiration in ms since epoch.
+   */
+  public void setExpiresInMSSinceEpoch(String expiresInMSSinceEpoch){
+    this.nextRefreshMSSinceEpoch = Long.parseLong(expiresInMSSinceEpoch);
+  }
+
+  /**
+   * Get next time we should refresh the token.
+   * 
+   * @return Next time since epoch we'll need to refresh the token.
+   */
+  public long getNextRefreshMSSinceEpoch() {
+    return nextRefreshMSSinceEpoch;
+  }
+  
+  /**
+   * Return true if the current token has expired or will expire within the
+   * EXPIRE_BUFFER_MS (to give ample wiggle room for the call to be made to
+   * the server).
+   */
+  public boolean shouldRefresh() {
+    long lowerLimit = nextRefreshMSSinceEpoch - EXPIRE_BUFFER_MS;
+    long currTime = timer.now();
+    return currTime > lowerLimit;
+  }
+  
+  /**
+   * The expires_in param from OAuth is in seconds-from-now.  Convert to
+   * milliseconds-from-epoch
+   */
+  static Long convertExpiresIn(Timer timer, String expiresInSecs) {
+    long expiresSecs = Long.parseLong(expiresInSecs);
+    long expiresMs = expiresSecs * 1000;
+    return timer.now() + expiresMs;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.java
new file mode 100644
index 0000000..b56dbde
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfCredentialBasedAccessTokenProvider.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.
+ */
+package org.apache.hadoop.hdfs.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Timer;
+
+import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
+
+/**
+ * Obtain an access token via a a credential (provided through the
+ * Configuration) using the 
+ * <a href="https://tools.ietf.org/html/rfc6749#section-4.4">
+ *   Client Credentials Grant workflow</a>.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ConfCredentialBasedAccessTokenProvider
+    extends CredentialBasedAccessTokenProvider {
+  private String credential;
+
+  public ConfCredentialBasedAccessTokenProvider() {
+  }
+
+  public ConfCredentialBasedAccessTokenProvider(Timer timer) {
+    super(timer);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    credential = notNull(conf, OAUTH_CREDENTIAL_KEY);
+  }
+
+  @Override
+  public String getCredential() {
+    if(credential == null) {
+      throw new IllegalArgumentException("Credential has not been " +
+          "provided in configuration");
+    }
+    
+    return credential;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
new file mode 100644
index 0000000..1e80451
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java
@@ -0,0 +1,146 @@
+/*
+ * 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.web.oauth2;
+
+import com.squareup.okhttp.OkHttpClient;
+import com.squareup.okhttp.Request;
+import com.squareup.okhttp.RequestBody;
+import com.squareup.okhttp.Response;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.util.Timer;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.REFRESH_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.URLENCODED;
+import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
+
+/**
+ * Supply a access token obtained via a refresh token (provided through the
+ * Configuration using the second half of the
+ * <a href="https://tools.ietf.org/html/rfc6749#section-4.1">
+ *   Authorization Code Grant workflow</a>.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ConfRefreshTokenBasedAccessTokenProvider
+    extends AccessTokenProvider {
+  
+  public static final String OAUTH_REFRESH_TOKEN_KEY
+      = "dfs.webhdfs.oauth2.refresh.token";
+  public static final String OAUTH_REFRESH_TOKEN_EXPIRES_KEY
+      = "dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch";
+
+  private AccessTokenTimer accessTokenTimer;
+  
+  private String accessToken;
+  
+  private String refreshToken;
+  
+  private String clientId;
+  
+  private String refreshURL;
+
+  
+  public ConfRefreshTokenBasedAccessTokenProvider() {
+    this.accessTokenTimer = new AccessTokenTimer();
+  }
+  
+  public ConfRefreshTokenBasedAccessTokenProvider(Timer timer) {
+    this.accessTokenTimer = new AccessTokenTimer(timer);
+  }
+  
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    refreshToken = notNull(conf, (OAUTH_REFRESH_TOKEN_KEY));
+    
+    accessTokenTimer.setExpiresInMSSinceEpoch(
+        notNull(conf, OAUTH_REFRESH_TOKEN_EXPIRES_KEY));
+
+    clientId = notNull(conf, OAUTH_CLIENT_ID_KEY);
+    refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY);
+    
+  }
+
+  @Override
+  public synchronized String getAccessToken() throws IOException {
+    if(accessTokenTimer.shouldRefresh()) {
+      refresh();
+    }
+    
+    return accessToken;
+  }
+  
+  void refresh() throws IOException {
+    try {
+      OkHttpClient client = new OkHttpClient();
+      client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+          TimeUnit.MILLISECONDS);
+      client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+                TimeUnit.MILLISECONDS);
+
+      String bodyString = Utils.postBody(GRANT_TYPE, REFRESH_TOKEN,
+          REFRESH_TOKEN, refreshToken,
+          CLIENT_ID, clientId);
+
+      RequestBody body = RequestBody.create(URLENCODED, bodyString);
+
+      Request request = new Request.Builder()
+          .url(refreshURL)
+          .post(body)
+          .build();
+      Response responseBody = client.newCall(request).execute();
+
+      if (responseBody.code() != HttpStatus.SC_OK) {
+        throw new IllegalArgumentException("Received invalid http response: "
+            + responseBody.code() + ", text = " + responseBody.toString());
+      }
+
+      ObjectMapper mapper = new ObjectMapper();
+      Map<?, ?> response = mapper.reader(Map.class)
+          .readValue(responseBody.body().string());
+
+
+      String newExpiresIn = response.get(EXPIRES_IN).toString();
+      accessTokenTimer.setExpiresIn(newExpiresIn);
+
+      accessToken = response.get(ACCESS_TOKEN).toString();
+    } catch (Exception e) {
+      throw new IOException("Exception while refreshing access token", e);
+    }
+  }
+  
+  public String getRefreshToken() {
+    return refreshToken;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
new file mode 100644
index 0000000..c058e05
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java
@@ -0,0 +1,135 @@
+/*
+ * 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.web.oauth2;
+
+import com.squareup.okhttp.OkHttpClient;
+import com.squareup.okhttp.Request;
+import com.squareup.okhttp.RequestBody;
+import com.squareup.okhttp.Response;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.util.Timer;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_CREDENTIALS;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_SECRET;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.URLENCODED;
+import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
+
+/**
+ * Obtain an access token via the credential-based OAuth2 workflow.  This
+ * abstract class requires only that implementations provide the credential,
+ * which the class then uses to obtain a refresh token.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class CredentialBasedAccessTokenProvider
+    extends AccessTokenProvider {
+  public static final String OAUTH_CREDENTIAL_KEY
+      = "dfs.webhdfs.oauth2.credential";
+  
+  private AccessTokenTimer timer;
+  
+  private String clientId;
+  
+  private String refreshURL;
+  
+  private String accessToken;
+  
+  private boolean initialCredentialObtained = false;
+
+  CredentialBasedAccessTokenProvider() {
+    this.timer = new AccessTokenTimer();
+  }
+  
+  CredentialBasedAccessTokenProvider(Timer timer) {
+    this.timer = new AccessTokenTimer(timer);
+  }
+  
+  abstract String getCredential();
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    clientId = notNull(conf, OAUTH_CLIENT_ID_KEY);
+    refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY);
+  }
+
+  @Override
+  public synchronized String getAccessToken() throws IOException {
+    if(timer.shouldRefresh() || !initialCredentialObtained) {
+      refresh();
+      initialCredentialObtained = true;
+    }
+    
+    return accessToken;
+  }
+  
+  void refresh() throws IOException {
+    try {
+      OkHttpClient client = new OkHttpClient();
+      client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+          TimeUnit.MILLISECONDS);
+      client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
+          TimeUnit.MILLISECONDS);
+
+      String bodyString = Utils.postBody(CLIENT_SECRET, getCredential(),
+          GRANT_TYPE, CLIENT_CREDENTIALS,
+          CLIENT_ID, clientId);
+
+      RequestBody body = RequestBody.create(URLENCODED, bodyString);
+
+      Request request = new Request.Builder()
+          .url(refreshURL)
+          .post(body)
+          .build();
+      Response responseBody = client.newCall(request).execute();
+
+      if (responseBody.code() != HttpStatus.SC_OK) {
+        throw new IllegalArgumentException("Received invalid http response: "
+            + responseBody.code() + ", text = " + responseBody.toString());
+      }
+
+      ObjectMapper mapper = new ObjectMapper();
+      Map<?, ?> response = mapper.reader(Map.class)
+          .readValue(responseBody.body().string());
+      
+      String newExpiresIn = response.get(EXPIRES_IN).toString();
+      timer.setExpiresIn(newExpiresIn);
+
+      accessToken = response.get(ACCESS_TOKEN).toString();
+
+    } catch (Exception e) {
+      throw new IOException("Unable to obtain access token from credential", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java
new file mode 100644
index 0000000..f334b24
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2ConnectionConfigurator.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hdfs.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
+
+/**
+ * Configure a connection to use OAuth2 authentication.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OAuth2ConnectionConfigurator implements ConnectionConfigurator {
+  
+  public static final String HEADER = "Bearer ";
+  
+  private final AccessTokenProvider accessTokenProvider;
+ 
+  private ConnectionConfigurator sslConfigurator = null;
+  
+  public OAuth2ConnectionConfigurator(Configuration conf) {
+    this(conf, null);
+  }
+  
+  @SuppressWarnings("unchecked")
+  public OAuth2ConnectionConfigurator(Configuration conf,
+                                      ConnectionConfigurator sslConfigurator) {
+    this.sslConfigurator = sslConfigurator;
+    
+    notNull(conf, ACCESS_TOKEN_PROVIDER_KEY);
+    
+    Class accessTokenProviderClass = conf.getClass(ACCESS_TOKEN_PROVIDER_KEY,
+        ConfCredentialBasedAccessTokenProvider.class,
+        AccessTokenProvider.class);
+    
+    accessTokenProvider = (AccessTokenProvider) ReflectionUtils
+        .newInstance(accessTokenProviderClass, conf);
+    accessTokenProvider.setConf(conf);
+  }
+  
+  @Override
+  public HttpURLConnection configure(HttpURLConnection conn)
+      throws IOException {
+    if(sslConfigurator != null) {
+      sslConfigurator.configure(conn);
+    }
+    
+    String accessToken = accessTokenProvider.getAccessToken();
+    
+    conn.setRequestProperty("AUTHORIZATION", HEADER + accessToken);
+    
+    return conn;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java
new file mode 100644
index 0000000..190a1f5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.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.web.oauth2;
+
+import com.squareup.okhttp.MediaType;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Sundry constants relating to OAuth2 within WebHDFS.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class OAuth2Constants {
+  private OAuth2Constants() { /** Private constructor. **/ }
+
+  public static final MediaType URLENCODED
+      = MediaType.parse("application/x-www-form-urlencoded; charset=utf-8");
+  
+  /* Constants for OAuth protocol */ 
+  public static final String ACCESS_TOKEN = "access_token";
+  public static final String BEARER = "bearer";
+  public static final String CLIENT_CREDENTIALS = "client_credentials";
+  public static final String CLIENT_ID = "client_id";
+  public static final String CLIENT_SECRET = "client_secret";
+  public static final String EXPIRES_IN = "expires_in";
+  public static final String GRANT_TYPE = "grant_type";
+  public static final String REFRESH_TOKEN = "refresh_token";
+  public static final String TOKEN_TYPE = "token_type";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.java
new file mode 100644
index 0000000..939798d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/Utils.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.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class Utils {
+  private Utils() { /* Private constructor */ }
+
+  public static String notNull(Configuration conf, String key) {
+    String value = conf.get(key);
+
+    if(value == null) {
+      throw new IllegalArgumentException("No value for " + key +
+          " found in conf file.");
+    }
+
+    return value;
+  }
+  
+  public static String postBody(String ... kv)
+      throws UnsupportedEncodingException {
+    if(kv.length % 2 != 0) {
+      throw new IllegalArgumentException("Arguments must be key value pairs");
+    }
+    StringBuilder sb = new StringBuilder();
+    int i = 0;
+    
+    while(i < kv.length) {
+      if(i > 0) {
+        sb.append("&");
+      }
+      sb.append(URLEncoder.encode(kv[i++], "UTF-8"));
+      sb.append("=");
+      sb.append(URLEncoder.encode(kv[i++], "UTF-8"));
+    }
+    
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
new file mode 100644
index 0000000..aeb581f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/**
+ * OAuth2-based WebHDFS authentication.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.hdfs.web.oauth2;
+
+import org.apache.hadoop.classification.InterfaceAudience;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6f46ea5..3382f81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -357,6 +357,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8131. Implement a space balanced block placement policy (Liu Shaohui
     via kihwal)
 
+    HDFS-8155. Support OAuth2 in WebHDFS. (jghoman)
+
   IMPROVEMENTS
 
     HDFS-2390. dfsadmin -setBalancerBandwidth does not validate -ve value

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 1a29ad3..d0c2dc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -213,6 +213,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>leveldbjni-all</artifactId>
       <version>1.8</version>
     </dependency>
+    <dependency>
+      <groupId>org.mock-server</groupId>
+      <artifactId>mockserver-netty</artifactId>
+      <version>3.9.2</version>
+      <scope>test</scope>
+    </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
       <groupId>org.bouncycastle</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index 20b9d73..d0a0fe0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -221,6 +221,31 @@ Below are examples using the `curl` command tool.
 
 See also: [Authentication for Hadoop HTTP web-consoles](../hadoop-common/HttpAuthentication.html)
 
+Additionally, WebHDFS supports OAuth2 on the client side. The Namenode and Datanodes do not currently support clients using OAuth2 but other backends that implement the WebHDFS REST interface may.
+
+WebHDFS supports two type of OAuth2 code grants (user-provided refresh and access token or user provided credential) by default and provides a pluggable mechanism for implementing other OAuth2 authentications per the [OAuth2 RFC](https://tools.ietf.org/html/rfc6749), or custom authentications.  When using either of the provided code grant mechanisms, the WebHDFS client will refresh the access token as necessary.
+
+OAuth2 should only be enabled for clients not running with Kerberos SPENGO.
+
+| OAuth2 code grant mechanism | Description | Value of `dfs.webhdfs.oauth2.access.token.provider` that implements code grant |
+|:---- |:---- |:----|
+| Authorization Code Grant | The user provides an initial access token and refresh token, which are then used to authenticate WebHDFS requests and obtain replacement access tokens, respectively. | org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider |
+| Client Credentials Grant | The user provides a credential which is used to obtain access tokens, which are then used to authenticate WebHDFS requests. | org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider |
+
+
+The following properties control OAuth2 authentication.
+
+| OAuth2 related property | Description |
+|:---- |:---- |
+| `dfs.webhdfs.oauth2.enabled` | Boolean to enable/disable OAuth2 authentication |
+| `dfs.webhdfs.oauth2.access.token.provider` | Class name of an implementation of `org.apache.hadoop.hdfs.web.oauth.AccessTokenProvider.`  Two are provided with the code, as described above, or the user may specify a user-provided implementation. The default value for this configuration key is the `ConfCredentialBasedAccessTokenProvider` implementation. |
+| `dfs.webhdfs.oauth2.client.id` | Client id used to obtain access token with either credential or refresh token |
+| `dfs.webhdfs.oauth2.refresh.url` | URL against which to post for obtaining bearer token with either credential or refresh token |
+| `dfs.webhdfs.oauth2.access.token` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Initial access token with which to authenticate |
+| `dfs.webhdfs.oauth2.refresh.token` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Initial refresh token to use to obtain new access tokens  |
+| `dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch` | (required if using ConfRefreshTokenBasedAccessTokenProvider) Access token expiration measured in milliseconds since Jan 1, 1970.  *Note this is a different value than provided by OAuth providers and has been munged as described in interface to be suitable for a client application*  |
+| `dfs.webhdfs.oauth2.credential` | (required if using ConfCredentialBasedAccessTokenProvider).  Credential used to obtain initial and subsequent access tokens. |
+
 Proxy Users
 -----------
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.java
new file mode 100644
index 0000000..e2f6230
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFSOAuth2.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.web;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.CredentialBasedAccessTokenProvider;
+import org.apache.hadoop.hdfs.web.oauth2.OAuth2ConnectionConfigurator;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockserver.client.server.MockServerClient;
+import org.mockserver.integration.ClientAndServer;
+import org.mockserver.model.Header;
+import org.mockserver.model.HttpRequest;
+import org.mockserver.model.HttpResponse;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE;
+import static org.junit.Assert.assertEquals;
+import static org.mockserver.integration.ClientAndServer.startClientAndServer;
+import static org.mockserver.matchers.Times.exactly;
+import static org.mockserver.model.HttpRequest.request;
+import static org.mockserver.model.HttpResponse.response;
+
+public class TestWebHDFSOAuth2 {
+  public static final Log LOG = LogFactory.getLog(TestWebHDFSOAuth2.class);
+
+  private ClientAndServer mockWebHDFS;
+  private ClientAndServer mockOAuthServer;
+
+  public final static int WEBHDFS_PORT = 7552;
+  public final static int OAUTH_PORT = 7553;
+
+  public final static Header CONTENT_TYPE_APPLICATION_JSON = new Header("Content-Type", "application/json");
+
+  public final static String AUTH_TOKEN = "0123456789abcdef";
+  public final static Header AUTH_TOKEN_HEADER = new Header("AUTHORIZATION", OAuth2ConnectionConfigurator.HEADER + AUTH_TOKEN);
+
+  @Before
+  public void startMockOAuthServer() {
+    mockOAuthServer = startClientAndServer(OAUTH_PORT);
+  }
+  @Before
+  public void startMockWebHDFSServer() {
+    System.setProperty("hadoop.home.dir", System.getProperty("user.dir"));
+
+    mockWebHDFS = startClientAndServer(WEBHDFS_PORT);
+  }
+
+  @Test
+  public void listStatusReturnsAsExpected() throws URISyntaxException, IOException {
+    MockServerClient mockWebHDFSServerClient = new MockServerClient("localhost", WEBHDFS_PORT);
+    MockServerClient mockOAuthServerClient = new MockServerClient("localhost", OAUTH_PORT);
+
+    HttpRequest oauthServerRequest = getOAuthServerMockRequest(mockOAuthServerClient);
+
+    HttpRequest fileSystemRequest = request()
+        .withMethod("GET")
+        .withPath(WebHdfsFileSystem.PATH_PREFIX + "/test1/test2")
+        .withHeader(AUTH_TOKEN_HEADER);
+
+    try {
+      mockWebHDFSServerClient.when(fileSystemRequest,
+          exactly(1)
+      )
+          .respond(
+              response()
+                  .withStatusCode(HttpStatus.SC_OK)
+                  .withHeaders(
+                      CONTENT_TYPE_APPLICATION_JSON
+                  )
+                  .withBody("{\n" +
+                      "  \"FileStatuses\":\n" +
+                      "  {\n" +
+                      "    \"FileStatus\":\n" +
+                      "    [\n" +
+                      "      {\n" +
+                      "        \"accessTime\"      : 1320171722771,\n" +
+                      "        \"blockSize\"       : 33554432,\n" +
+                      "        \"group\"           : \"supergroup\",\n" +
+                      "        \"length\"          : 24930,\n" +
+                      "        \"modificationTime\": 1320171722771,\n" +
+                      "        \"owner\"           : \"webuser\",\n" +
+                      "        \"pathSuffix\"      : \"a.patch\",\n" +
+                      "        \"permission\"      : \"644\",\n" +
+                      "        \"replication\"     : 1,\n" +
+                      "        \"type\"            : \"FILE\"\n" +
+                      "      },\n" +
+                      "      {\n" +
+                      "        \"accessTime\"      : 0,\n" +
+                      "        \"blockSize\"       : 0,\n" +
+                      "        \"group\"           : \"supergroup\",\n" +
+                      "        \"length\"          : 0,\n" +
+                      "        \"modificationTime\": 1320895981256,\n" +
+                      "        \"owner\"           : \"szetszwo\",\n" +
+                      "        \"pathSuffix\"      : \"bar\",\n" +
+                      "        \"permission\"      : \"711\",\n" +
+                      "        \"replication\"     : 0,\n" +
+                      "        \"type\"            : \"DIRECTORY\"\n" +
+                      "      }\n" +
+                      "    ]\n" +
+                      "  }\n" +
+                      "}\n")
+          );
+
+      FileSystem fs = new WebHdfsFileSystem();
+      Configuration conf = getConfiguration();
+      conf.set(OAUTH_REFRESH_URL_KEY, "http://localhost:" + OAUTH_PORT + "/refresh");
+      conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY, "credential");
+
+      URI uri = new URI("webhdfs://localhost:" + WEBHDFS_PORT);
+      fs.initialize(uri, conf);
+
+      FileStatus[] ls = fs.listStatus(new Path("/test1/test2"));
+
+      mockOAuthServer.verify(oauthServerRequest);
+      mockWebHDFSServerClient.verify(fileSystemRequest);
+
+      assertEquals(2, ls.length);
+      assertEquals("a.patch", ls[0].getPath().getName());
+      assertEquals("bar", ls[1].getPath().getName());
+
+      fs.close();
+    } finally {
+      mockWebHDFSServerClient.clear(fileSystemRequest);
+      mockOAuthServerClient.clear(oauthServerRequest);
+    }
+  }
+
+  private HttpRequest getOAuthServerMockRequest(MockServerClient mockServerClient) throws IOException {
+    HttpRequest expectedRequest = request()
+        .withMethod("POST")
+        .withPath("/refresh")
+        .withBody("client_secret=credential&grant_type=client_credentials&client_id=MY_CLIENTID");
+    
+    Map<String, Object> map = new TreeMap<>();
+    
+    map.put(EXPIRES_IN, "0987654321");
+    map.put(TOKEN_TYPE, "bearer");
+    map.put(ACCESS_TOKEN, AUTH_TOKEN);
+
+    ObjectMapper mapper = new ObjectMapper();
+    
+    HttpResponse resp = response()
+        .withStatusCode(HttpStatus.SC_OK)
+        .withHeaders(
+            CONTENT_TYPE_APPLICATION_JSON
+        )
+        .withBody(mapper.writeValueAsString(map));
+
+    mockServerClient
+        .when(expectedRequest, exactly(1))
+        .respond(resp);
+
+    return expectedRequest;
+  }
+
+  public Configuration getConfiguration() {
+    Configuration conf = new Configuration();
+
+    // Configs for OAuth2
+    conf.setBoolean(HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY, true);
+    conf.set(OAUTH_CLIENT_ID_KEY, "MY_CLIENTID");
+
+    conf.set(ACCESS_TOKEN_PROVIDER_KEY,
+        ConfCredentialBasedAccessTokenProvider.class.getName());
+
+    return conf;
+
+  }
+
+  @After
+  public void stopMockWebHDFSServer() {
+      mockWebHDFS.stop();
+  }
+
+  @After
+  public void stopMockOAuthServer() {
+    mockOAuthServer.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.java
new file mode 100644
index 0000000..c387b1e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestAccessTokenTimer.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.web.oauth2;
+
+import org.apache.hadoop.util.Timer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestAccessTokenTimer {
+  @Test
+  public void expireConversionWorks() {
+    Timer mockTimer = mock(Timer.class);
+    when(mockTimer.now())
+        .thenReturn(5l);
+    
+    AccessTokenTimer timer = new AccessTokenTimer(mockTimer);
+    
+    timer.setExpiresIn("3");
+    assertEquals(3005, timer.getNextRefreshMSSinceEpoch());
+    
+    assertTrue(timer.shouldRefresh());
+  }
+  
+  @Test
+  public void shouldRefreshIsCorrect() {
+    Timer mockTimer = mock(Timer.class);
+    when(mockTimer.now())
+        .thenReturn(500l)
+        .thenReturn(1000000l + 500l);
+    
+    AccessTokenTimer timer = new AccessTokenTimer(mockTimer);
+    
+    timer.setExpiresInMSSinceEpoch("1000000");
+    
+    assertFalse(timer.shouldRefresh());
+    assertTrue(timer.shouldRefresh());
+    
+    verify(mockTimer, times(2)).now();
+  } 
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java
new file mode 100644
index 0000000..c259b30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestClientCredentialTimeBasedTokenRefresher.java
@@ -0,0 +1,138 @@
+/*
+ * 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.web.oauth2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Timer;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Test;
+import org.mockserver.client.server.MockServerClient;
+import org.mockserver.integration.ClientAndServer;
+import org.mockserver.model.Header;
+import org.mockserver.model.HttpRequest;
+import org.mockserver.model.HttpResponse;
+import org.mockserver.model.Parameter;
+import org.mockserver.model.ParameterBody;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ACCESS_TOKEN_PROVIDER_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_CREDENTIALS;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_SECRET;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockserver.integration.ClientAndServer.startClientAndServer;
+import static org.mockserver.matchers.Times.exactly;
+import static org.mockserver.model.HttpRequest.request;
+import static org.mockserver.model.HttpResponse.response;
+
+public class TestClientCredentialTimeBasedTokenRefresher {
+  public final static Header CONTENT_TYPE_APPLICATION_JSON
+      = new Header("Content-Type", "application/json");
+
+  public final static String CLIENT_ID_FOR_TESTING = "joebob";
+
+  public Configuration buildConf(String credential, String tokenExpires,
+                                 String clientId, String refreshURL) {
+    // Configurations are simple enough that it's not worth mocking them out.
+    Configuration conf = new Configuration();
+    conf.set(CredentialBasedAccessTokenProvider.OAUTH_CREDENTIAL_KEY,
+        credential);
+    conf.set(ACCESS_TOKEN_PROVIDER_KEY,
+        ConfCredentialBasedAccessTokenProvider.class.getName());
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId);
+    conf.set(OAUTH_REFRESH_URL_KEY, refreshURL);
+    return conf;
+  }
+
+  @Test
+  public void refreshUrlIsCorrect() throws IOException {
+    final int PORT = 7552;
+    final String REFRESH_ADDRESS = "http://localhost:" + PORT + "/refresh";
+
+    long tokenExpires = 0;
+
+    Configuration conf = buildConf("myreallycoolcredential",
+        Long.toString(tokenExpires),
+        CLIENT_ID_FOR_TESTING,
+        REFRESH_ADDRESS);
+
+    Timer mockTimer = mock(Timer.class);
+    when(mockTimer.now()).thenReturn(tokenExpires + 1000l);
+
+    AccessTokenProvider credProvider =
+        new ConfCredentialBasedAccessTokenProvider(mockTimer);
+    credProvider.setConf(conf);
+    
+    // Build mock server to receive refresh request
+    ClientAndServer mockServer  = startClientAndServer(PORT);
+
+    HttpRequest expectedRequest = request()
+        .withMethod("POST")
+        .withPath("/refresh")
+        .withBody( 
+        // Note, OkHttp does not sort the param values, so we need to do
+        // it ourselves via the ordering provided to ParameterBody...
+            ParameterBody.params(
+                Parameter.param(CLIENT_SECRET, "myreallycoolcredential"),
+                Parameter.param(GRANT_TYPE, CLIENT_CREDENTIALS),
+                Parameter.param(CLIENT_ID, CLIENT_ID_FOR_TESTING)
+                ));
+
+    MockServerClient mockServerClient = new MockServerClient("localhost", PORT);
+
+    // https://tools.ietf.org/html/rfc6749#section-5.1
+    Map<String, Object> map = new TreeMap<>();
+    
+    map.put(EXPIRES_IN, "0987654321");
+    map.put(TOKEN_TYPE, "bearer");
+    map.put(ACCESS_TOKEN, "new access token");
+
+    ObjectMapper mapper = new ObjectMapper();
+    
+    HttpResponse resp = response()
+        .withStatusCode(HttpStatus.SC_OK)
+        .withHeaders(
+            CONTENT_TYPE_APPLICATION_JSON
+        )
+        .withBody(mapper.writeValueAsString(map));
+
+    mockServerClient
+        .when(expectedRequest, exactly(1))
+        .respond(resp);
+
+    assertEquals("new access token", credProvider.getAccessToken());
+
+    mockServerClient.verify(expectedRequest);
+
+    mockServerClient.clear(expectedRequest);
+    mockServer.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/837fb75e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java
new file mode 100644
index 0000000..889ad0e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/oauth2/TestRefreshTokenTimeBasedTokenRefresher.java
@@ -0,0 +1,138 @@
+/*
+ * 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.web.oauth2;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Timer;
+import org.apache.http.HttpStatus;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Test;
+import org.mockserver.client.server.MockServerClient;
+import org.mockserver.integration.ClientAndServer;
+import org.mockserver.model.Header;
+import org.mockserver.model.HttpRequest;
+import org.mockserver.model.HttpResponse;
+import org.mockserver.model.Parameter;
+import org.mockserver.model.ParameterBody;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_EXPIRES_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.ConfRefreshTokenBasedAccessTokenProvider.OAUTH_REFRESH_TOKEN_KEY;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.BEARER;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.CLIENT_ID;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.EXPIRES_IN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.GRANT_TYPE;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.REFRESH_TOKEN;
+import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.TOKEN_TYPE;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockserver.integration.ClientAndServer.startClientAndServer;
+import static org.mockserver.matchers.Times.exactly;
+import static org.mockserver.model.HttpRequest.request;
+import static org.mockserver.model.HttpResponse.response;
+
+public class TestRefreshTokenTimeBasedTokenRefresher {
+
+  public final static Header CONTENT_TYPE_APPLICATION_JSON
+      = new Header("Content-Type", "application/json");
+
+  public Configuration buildConf(String refreshToken, String tokenExpires,
+                                 String clientId, String refreshURL) {
+    // Configurations are simple enough that it's not worth mocking them out.
+    Configuration conf = new Configuration();
+    conf.set(OAUTH_REFRESH_TOKEN_KEY, refreshToken);
+    conf.set(OAUTH_REFRESH_TOKEN_EXPIRES_KEY, tokenExpires);
+    conf.set(OAUTH_CLIENT_ID_KEY, clientId);
+    conf.set(OAUTH_REFRESH_URL_KEY, refreshURL);
+
+    return conf;
+  }
+
+  @Test
+  public void refreshUrlIsCorrect() throws IOException {
+    final int PORT = 7552;
+    final String REFRESH_ADDRESS = "http://localhost:" + PORT + "/refresh";
+
+    long tokenExpires = 0;
+
+    Configuration conf = buildConf("refresh token key",
+        Long.toString(tokenExpires),
+        "joebob",
+        REFRESH_ADDRESS);
+
+    Timer mockTimer = mock(Timer.class);
+    when(mockTimer.now()).thenReturn(tokenExpires + 1000l);
+
+    AccessTokenProvider tokenProvider =
+        new ConfRefreshTokenBasedAccessTokenProvider(mockTimer);
+    tokenProvider.setConf(conf);
+
+    // Build mock server to receive refresh request
+
+    ClientAndServer mockServer  = startClientAndServer(PORT);
+
+    HttpRequest expectedRequest = request()
+        .withMethod("POST")
+        .withPath("/refresh")
+        // Note, OkHttp does not sort the param values, so we need to
+        // do it ourselves via the ordering provided to ParameterBody...
+        .withBody(
+            ParameterBody.params(
+                Parameter.param(CLIENT_ID, "joebob"),
+                Parameter.param(GRANT_TYPE, REFRESH_TOKEN),
+                Parameter.param(REFRESH_TOKEN, "refresh token key")));
+
+    MockServerClient mockServerClient = new MockServerClient("localhost", PORT);
+
+    // https://tools.ietf.org/html/rfc6749#section-5.1
+    Map<String, Object> map = new TreeMap<>();
+
+    map.put(EXPIRES_IN, "0987654321");
+    map.put(TOKEN_TYPE, BEARER);
+    map.put(ACCESS_TOKEN, "new access token");
+
+    ObjectMapper mapper = new ObjectMapper();
+    
+    HttpResponse resp = response()
+        .withStatusCode(HttpStatus.SC_OK)
+        .withHeaders(
+            CONTENT_TYPE_APPLICATION_JSON
+        )
+        .withBody(mapper.writeValueAsString(map));
+
+    mockServerClient
+        .when(expectedRequest, exactly(1))
+        .respond(resp);
+
+    assertEquals("new access token", tokenProvider.getAccessToken());
+
+    mockServerClient.verify(expectedRequest);
+
+    mockServerClient.clear(expectedRequest);
+    mockServer.stop();
+  }
+
+}


[22/50] [abbrv] hadoop git commit: YARN-3893. Both RM in active state when Admin#transitionToActive failure from refeshAll() (Bibin A Chundatt via rohithsharmaks)

Posted by wa...@apache.org.
YARN-3893. Both RM in active state when Admin#transitionToActive failure from refeshAll() (Bibin A Chundatt via rohithsharmaks)


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

Branch: refs/heads/YARN-1197
Commit: 7d6687fe76f6152a577ff2298c358dd30fce41fb
Parents: 095ab9a
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Wed Sep 2 15:22:48 2015 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Wed Sep 2 15:22:48 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../server/resourcemanager/AdminService.java    | 20 ++++--
 .../resourcemanager/RMFatalEventType.java       |  5 +-
 .../yarn/server/resourcemanager/TestRMHA.java   | 71 ++++++++++++++++++++
 4 files changed, 94 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d6687fe/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index aade2d7..13fe9b0 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -868,6 +868,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3857: Memory leak in ResourceManager with SIMPLE mode.
     (mujunchao via zxu)
 
+    YARN-3893. Both RM in active state when Admin#transitionToActive failure 
+    from refeshAll() (Bibin A Chundatt via rohithsharmaks)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d6687fe/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 20c9800..d96ed8c 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
@@ -297,6 +297,7 @@ public class AdminService extends CompositeService implements
     }
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public synchronized void transitionToActive(
       HAServiceProtocol.StateChangeRequestInfo reqInfo) throws IOException {
@@ -312,10 +313,6 @@ public class AdminService extends CompositeService implements
     checkHaStateChange(reqInfo);
     try {
       rm.transitionToActive();
-      // call all refresh*s for active RM to get the updated configurations.
-      refreshAll();
-      RMAuditLogger.logSuccess(user.getShortUserName(),
-          "transitionToActive", "RMHAProtocolService");
     } catch (Exception e) {
       RMAuditLogger.logFailure(user.getShortUserName(), "transitionToActive",
           "", "RMHAProtocolService",
@@ -323,6 +320,21 @@ public class AdminService extends CompositeService implements
       throw new ServiceFailedException(
           "Error when transitioning to Active mode", e);
     }
+    try {
+      // call all refresh*s for active RM to get the updated configurations.
+      refreshAll();
+    } catch (Exception e) {
+      LOG.error("RefreshAll failed so firing fatal event", e);
+      rmContext
+          .getDispatcher()
+          .getEventHandler()
+          .handle(
+          new RMFatalEvent(RMFatalEventType.TRANSITION_TO_ACTIVE_FAILED, e));
+      throw new ServiceFailedException(
+          "Error on refreshAll during transistion to Active", e);
+    }
+    RMAuditLogger.logSuccess(user.getShortUserName(), "transitionToActive",
+        "RMHAProtocolService");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d6687fe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.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/RMFatalEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
index 789c018..87cc496 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMFatalEventType.java
@@ -26,5 +26,8 @@ public enum RMFatalEventType {
   STATE_STORE_OP_FAILED,
 
   // Source <- Embedded Elector
-  EMBEDDED_ELECTOR_FAILED
+  EMBEDDED_ELECTOR_FAILED,
+
+  // Source <- Admin Service
+  TRANSITION_TO_ACTIVE_FAILED
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d6687fe/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 0200e85..62cfe84 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
@@ -43,6 +43,8 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -52,6 +54,7 @@ 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.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
@@ -577,6 +580,56 @@ public class TestRMHA {
     assertEquals(0, rm.getRMContext().getRMApps().size());
   }
 
+  @Test(timeout = 90000)
+  public void testTransitionedToActiveRefreshFail() throws Exception {
+    configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
+    YarnConfiguration conf = new YarnConfiguration(configuration);
+    configuration = new CapacitySchedulerConfiguration(conf);
+    rm = new MockRM(configuration) {
+      @Override
+      protected AdminService createAdminService() {
+        return new AdminService(this, getRMContext()) {
+          @Override
+          protected void setConfig(Configuration conf) {
+            super.setConfig(configuration);
+          }
+        };
+      }
+
+      @Override
+      protected Dispatcher createDispatcher() {
+        return new FailFastDispatcher();
+      }
+    };
+
+    rm.init(configuration);
+    rm.start();
+    final StateChangeRequestInfo requestInfo =
+        new StateChangeRequestInfo(
+            HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+
+    configuration.set("yarn.scheduler.capacity.root.default.capacity", "100");
+    rm.adminService.transitionToStandby(requestInfo);
+    assertEquals(HAServiceState.STANDBY, rm.getRMContext().getHAServiceState());
+    configuration.set("yarn.scheduler.capacity.root.default.capacity", "200");
+    try {
+      rm.adminService.transitionToActive(requestInfo);
+    } catch (Exception e) {
+      assertTrue("Error on refreshAll during transistion to Active".contains(e
+          .getMessage()));
+    }
+    FailFastDispatcher dispatcher =
+        ((FailFastDispatcher) rm.rmContext.getDispatcher());
+    dispatcher.await();
+    assertEquals(1, dispatcher.getEventCount());
+    // Making correct conf and check the state
+    configuration.set("yarn.scheduler.capacity.root.default.capacity", "100");
+    rm.adminService.transitionToActive(requestInfo);
+    assertEquals(HAServiceState.ACTIVE, rm.getRMContext().getHAServiceState());
+    rm.adminService.transitionToStandby(requestInfo);
+    assertEquals(HAServiceState.STANDBY, rm.getRMContext().getHAServiceState());
+  }
+
   public void innerTestHAWithRMHostName(boolean includeBindHost) {
     //this is run two times, with and without a bind host configured
     if (includeBindHost) {
@@ -713,4 +766,22 @@ public class TestRMHA {
       return this.stopped;
     }
   }
+
+  class FailFastDispatcher extends DrainDispatcher {
+    int eventreceived = 0;
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    protected void dispatch(Event event) {
+      if (event.getType() == RMFatalEventType.TRANSITION_TO_ACTIVE_FAILED) {
+        eventreceived++;
+      } else {
+        super.dispatch(event);
+      }
+    }
+
+    public int getEventCount() {
+      return eventreceived;
+    }
+  }
 }


[21/50] [abbrv] hadoop git commit: YARN-4073. Removed unused ApplicationACLsManager in ContainerManagerImpl constructor. (Naganarasimha G R via rohithsharmaks)

Posted by wa...@apache.org.
YARN-4073. Removed unused ApplicationACLsManager in ContainerManagerImpl constructor. (Naganarasimha G R via rohithsharmaks)


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

Branch: refs/heads/YARN-1197
Commit: 095ab9ab5fa5f415fba41fee81b6008cf0455af8
Parents: 65ccf2b
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Wed Sep 2 14:13:33 2015 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Wed Sep 2 14:13:33 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                               | 3 +++
 .../apache/hadoop/yarn/server/nodemanager/NodeManager.java    | 2 +-
 .../nodemanager/containermanager/ContainerManagerImpl.java    | 6 +-----
 .../hadoop/yarn/server/nodemanager/DummyContainerManager.java | 7 ++-----
 .../apache/hadoop/yarn/server/nodemanager/TestEventFlow.java  | 3 +--
 .../hadoop/yarn/server/nodemanager/TestNodeManagerResync.java | 2 +-
 .../hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java | 6 +++---
 .../containermanager/BaseContainerManagerTest.java            | 3 +--
 .../nodemanager/containermanager/TestContainerManager.java    | 5 ++---
 .../containermanager/TestContainerManagerRecovery.java        | 3 +--
 .../yarn/server/nodemanager/containermanager/TestNMProxy.java | 3 +--
 11 files changed, 17 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b33111c..aade2d7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -807,6 +807,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4082. Container shouldn't be killed when node's label updated.
     (Wangda Tan via vvasudev)
 
+    YARN-4073. Removed unused ApplicationACLsManager in ContainerManagerImpl constructor.
+    (Naganarasimha G R via rohithsharmaks)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 68820a7..3cf9f1a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -166,7 +166,7 @@ public class NodeManager extends CompositeService
       NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager,
       LocalDirsHandlerService dirsHandler) {
     return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-      metrics, aclsManager, dirsHandler);
+      metrics, dirsHandler);
   }
 
   protected WebServer createWebServer(Context nmContext,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 494fa8f..68c7f2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -134,7 +134,6 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.Re
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider;
-import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -167,7 +166,6 @@ public class ContainerManagerImpl extends CompositeService implements
 
   protected LocalDirsHandlerService dirsHandler;
   protected final AsyncDispatcher dispatcher;
-  private final ApplicationACLsManager aclsManager;
 
   private final DeletionService deletionService;
   private AtomicBoolean blockNewContainerRequests = new AtomicBoolean(false);
@@ -179,8 +177,7 @@ public class ContainerManagerImpl extends CompositeService implements
 
   public ContainerManagerImpl(Context context, ContainerExecutor exec,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
-      NodeManagerMetrics metrics, ApplicationACLsManager aclsManager,
-      LocalDirsHandlerService dirsHandler) {
+      NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) {
     super(ContainerManagerImpl.class.getName());
     this.context = context;
     this.dirsHandler = dirsHandler;
@@ -198,7 +195,6 @@ public class ContainerManagerImpl extends CompositeService implements
     addService(containersLauncher);
 
     this.nodeStatusUpdater = nodeStatusUpdater;
-    this.aclsManager = aclsManager;
 
     // Start configurable services
     auxiliaryServices = new AuxServices();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.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/DummyContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
index f872a55..349340b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
-import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 
 public class DummyContainerManager extends ContainerManagerImpl {
 
@@ -63,11 +62,9 @@ public class DummyContainerManager extends ContainerManagerImpl {
   
   public DummyContainerManager(Context context, ContainerExecutor exec,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
-      NodeManagerMetrics metrics,
-      ApplicationACLsManager applicationACLsManager,
-      LocalDirsHandlerService dirsHandler) {
+      NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) {
     super(context, exec, deletionContext, nodeStatusUpdater, metrics,
-      applicationACLsManager, dirsHandler);
+        dirsHandler);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.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/TestEventFlow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
index c878e5c..3dc62bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
-import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.junit.Test;
 
 
@@ -128,7 +127,7 @@ public class TestEventFlow {
 
     DummyContainerManager containerManager =
         new DummyContainerManager(context, exec, del, nodeStatusUpdater,
-          metrics, new ApplicationACLsManager(conf), dirsHandler);
+          metrics, dirsHandler);
     nodeStatusUpdater.init(conf);
     ((NMContext)context).setContainerManager(containerManager);
     nodeStatusUpdater.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index 611e671..c22d475 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -435,7 +435,7 @@ public class TestNodeManagerResync {
         NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager,
         LocalDirsHandlerService dirsHandler) {
       return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-        metrics, aclsManager, dirsHandler){
+        metrics, dirsHandler){
         @Override
         public void setBlockNewContainerRequests(
             boolean blockNewContainerRequests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index a9ef72f..3c0368b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -1160,7 +1160,7 @@ public class TestNodeStatusUpdater {
           ApplicationACLsManager aclsManager,
           LocalDirsHandlerService dirsHandler) {
         return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-            metrics, aclsManager, dirsHandler) {
+            metrics, dirsHandler) {
 
           @Override
           public void cleanUpApplicationsOnNMShutDown() {
@@ -1373,7 +1373,7 @@ public class TestNodeStatusUpdater {
           ApplicationACLsManager aclsManager,
           LocalDirsHandlerService diskhandler) {
         return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-          metrics, aclsManager, diskhandler) {
+          metrics, diskhandler) {
           @Override
           protected void serviceStart() {
             // Simulating failure of starting RPC server
@@ -1523,7 +1523,7 @@ public class TestNodeStatusUpdater {
           ApplicationACLsManager aclsManager,
           LocalDirsHandlerService dirsHandler) {
         return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-            metrics, aclsManager, dirsHandler) {
+            metrics, dirsHandler) {
 
           @Override
           public void cleanUpApplicationsOnNMShutDown() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.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/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index a8e723d..2810662 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -192,7 +191,7 @@ public abstract class BaseContainerManagerTest {
       createContainerManager(DeletionService delSrvc) {
     
     return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
-      metrics, new ApplicationACLsManager(conf), dirsHandler) {
+      metrics, dirsHandler) {
       @Override
       public void
           setBlockNewContainerRequests(boolean blockNewContainerRequests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 7bdfdfb..e508424 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -80,7 +80,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
@@ -116,7 +115,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   protected ContainerManagerImpl
       createContainerManager(DeletionService delSrvc) {
     return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
-      metrics, new ApplicationACLsManager(conf), dirsHandler) {
+      metrics, dirsHandler) {
       @Override
       public void
           setBlockNewContainerRequests(boolean blockNewContainerRequests) {
@@ -801,7 +800,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   public void testNullTokens() throws Exception {
     ContainerManagerImpl cMgrImpl =
         new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
-        metrics, new ApplicationACLsManager(conf), dirsHandler);
+        metrics, dirsHandler);
     String strExceptionMsg = "";
     try {
       cMgrImpl.authorizeStartRequest(null, new ContainerTokenIdentifier());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.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/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 781950e..4d0aacd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -394,8 +394,7 @@ public class TestContainerManagerRecovery {
 
     return new ContainerManagerImpl(context,
         mock(ContainerExecutor.class), mock(DeletionService.class),
-        mock(NodeStatusUpdater.class), metrics,
-        context.getApplicationACLsManager(), null) {
+        mock(NodeStatusUpdater.class), metrics, null) {
           @Override
           protected LogHandler createLogHandler(Configuration conf,
               Context context, DeletionService deletionService) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/095ab9ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.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/TestNMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
index 102c9c6..152b92c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
-import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
@@ -67,7 +66,7 @@ public class TestNMProxy extends BaseContainerManagerTest {
   protected ContainerManagerImpl
       createContainerManager(DeletionService delSrvc) {
     return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
-      metrics, new ApplicationACLsManager(conf), dirsHandler) {
+      metrics, dirsHandler) {
 
       @Override
       public StartContainersResponse startContainers(


[48/50] [abbrv] hadoop git commit: YARN-1643. Make ContainersMonitor support changing monitoring size of an allocated container. Contributed by Meng Ding and Wangda Tan

Posted by wa...@apache.org.
YARN-1643. Make ContainersMonitor support changing monitoring size of an allocated container. Contributed by Meng Ding and Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: 5d70f93831c472ba82d4699a66b33c40780facf6
Parents: 508da11
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 5 15:19:33 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:04 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../monitor/ContainersMonitorImpl.java          | 207 ++++++++++------
 .../TestContainerManagerWithLCE.java            |  11 +
 .../containermanager/TestContainerManager.java  |  96 +++++++
 .../monitor/MockResourceCalculatorPlugin.java   |  69 ++++++
 .../MockResourceCalculatorProcessTree.java      |  57 +++++
 .../TestContainersMonitorResourceChange.java    | 248 +++++++++++++++++++
 7 files changed, 615 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 49eb7b9..a524bbc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -202,6 +202,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3867. ContainerImpl changes to support container resizing. (Meng Ding 
     via jianhe)
 
+    YARN-1643. Make ContainersMonitor support changing monitoring size of an
+    allocated container. (Meng Ding and Wangda Tan)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index afb51ad..b3839d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -18,13 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,12 +30,14 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 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.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
@@ -56,16 +56,16 @@ public class ContainersMonitorImpl extends AbstractService implements
   private boolean containerMetricsEnabled;
   private long containerMetricsPeriodMs;
 
-  final List<ContainerId> containersToBeRemoved;
-  final Map<ContainerId, ProcessTreeInfo> containersToBeAdded;
-  Map<ContainerId, ProcessTreeInfo> trackingContainers =
-      new HashMap<ContainerId, ProcessTreeInfo>();
+  @VisibleForTesting
+  final Map<ContainerId, ProcessTreeInfo> trackingContainers =
+      new ConcurrentHashMap<>();
 
-  final ContainerExecutor containerExecutor;
+  private final ContainerExecutor containerExecutor;
   private final Dispatcher eventDispatcher;
   private final Context context;
   private ResourceCalculatorPlugin resourceCalculatorPlugin;
   private Configuration conf;
+  private static float vmemRatio;
   private Class<? extends ResourceCalculatorProcessTree> processTreeClass;
 
   private long maxVmemAllottedForContainers = UNKNOWN_MEMORY_LIMIT;
@@ -82,6 +82,8 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   private ResourceUtilization containersUtilization;
 
+  private volatile boolean stopped = false;
+
   public ContainersMonitorImpl(ContainerExecutor exec,
       AsyncDispatcher dispatcher, Context context) {
     super("containers-monitor");
@@ -90,8 +92,6 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.eventDispatcher = dispatcher;
     this.context = context;
 
-    this.containersToBeAdded = new HashMap<ContainerId, ProcessTreeInfo>();
-    this.containersToBeRemoved = new ArrayList<ContainerId>();
     this.monitoringThread = new MonitoringThread();
 
     this.containersUtilization = ResourceUtilization.newInstance(0, 0, 0.0f);
@@ -140,7 +140,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.maxVCoresAllottedForContainers = configuredVCoresForContainers;
 
     // ///////// Virtual memory configuration //////
-    float vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
+    vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
         YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
     Preconditions.checkArgument(vmemRatio > 0.99f,
         YarnConfiguration.NM_VMEM_PMEM_RATIO + " should be at least 1.0");
@@ -218,6 +218,7 @@ public class ContainersMonitorImpl extends AbstractService implements
   @Override
   protected void serviceStop() throws Exception {
     if (containersMonitorEnabled) {
+      stopped = true;
       this.monitoringThread.interrupt();
       try {
         this.monitoringThread.join();
@@ -228,7 +229,8 @@ public class ContainersMonitorImpl extends AbstractService implements
     super.serviceStop();
   }
 
-  private static class ProcessTreeInfo {
+  @VisibleForTesting
+  static class ProcessTreeInfo {
     private ContainerId containerId;
     private String pid;
     private ResourceCalculatorProcessTree pTree;
@@ -267,26 +269,43 @@ public class ContainersMonitorImpl extends AbstractService implements
       this.pTree = pTree;
     }
 
-    public long getVmemLimit() {
+    /**
+     * @return Virtual memory limit for the process tree in bytes
+     */
+    public synchronized long getVmemLimit() {
       return this.vmemLimit;
     }
 
     /**
      * @return Physical memory limit for the process tree in bytes
      */
-    public long getPmemLimit() {
+    public synchronized long getPmemLimit() {
       return this.pmemLimit;
     }
 
     /**
-     * Return the number of cpu vcores assigned
-     * @return
+     * @return Number of cpu vcores assigned
      */
-    public int getCpuVcores() {
+    public synchronized int getCpuVcores() {
       return this.cpuVcores;
     }
-  }
 
+    /**
+     * Set resource limit for enforcement
+     * @param pmemLimit
+     *          Physical memory limit for the process tree in bytes
+     * @param vmemLimit
+     *          Virtual memory limit for the process tree in bytes
+     * @param cpuVcores
+     *          Number of cpu vcores assigned
+     */
+    public synchronized void setResourceLimit(
+        long pmemLimit, long vmemLimit, int cpuVcores) {
+      this.pmemLimit = pmemLimit;
+      this.vmemLimit = vmemLimit;
+      this.cpuVcores = cpuVcores;
+    }
+  }
 
   /**
    * Check whether a container's process tree's current memory usage is over
@@ -359,8 +378,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     @Override
     public void run() {
 
-      while (true) {
-
+      while (!stopped && !Thread.currentThread().isInterrupted()) {
         // Print the processTrees for debugging.
         if (LOG.isDebugEnabled()) {
           StringBuilder tmp = new StringBuilder("[ ");
@@ -372,31 +390,6 @@ public class ContainersMonitorImpl extends AbstractService implements
               + tmp.substring(0, tmp.length()) + "]");
         }
 
-        // Add new containers
-        synchronized (containersToBeAdded) {
-          for (Entry<ContainerId, ProcessTreeInfo> entry : containersToBeAdded
-              .entrySet()) {
-            ContainerId containerId = entry.getKey();
-            ProcessTreeInfo processTreeInfo = entry.getValue();
-            LOG.info("Starting resource-monitoring for " + containerId);
-            trackingContainers.put(containerId, processTreeInfo);
-          }
-          containersToBeAdded.clear();
-        }
-
-        // Remove finished containers
-        synchronized (containersToBeRemoved) {
-          for (ContainerId containerId : containersToBeRemoved) {
-            if (containerMetricsEnabled) {
-              ContainerMetrics.forContainer(
-                  containerId, containerMetricsPeriodMs).finished();
-            }
-            trackingContainers.remove(containerId);
-            LOG.info("Stopping resource-monitoring for " + containerId);
-          }
-          containersToBeRemoved.clear();
-        }
-
         // Temporary structure to calculate the total resource utilization of
         // the containers
         ResourceUtilization trackedContainersUtilization  =
@@ -408,10 +401,8 @@ public class ContainersMonitorImpl extends AbstractService implements
         long pmemByAllContainers = 0;
         long cpuUsagePercentPerCoreByAllContainers = 0;
         long cpuUsageTotalCoresByAllContainers = 0;
-        for (Iterator<Map.Entry<ContainerId, ProcessTreeInfo>> it =
-            trackingContainers.entrySet().iterator(); it.hasNext();) {
-
-          Map.Entry<ContainerId, ProcessTreeInfo> entry = it.next();
+        for (Entry<ContainerId, ProcessTreeInfo> entry : trackingContainers
+            .entrySet()) {
           ContainerId containerId = entry.getKey();
           ProcessTreeInfo ptInfo = entry.getValue();
           try {
@@ -435,11 +426,6 @@ public class ContainersMonitorImpl extends AbstractService implements
                 if (containerMetricsEnabled) {
                   ContainerMetrics usageMetrics = ContainerMetrics
                       .forContainer(containerId, containerMetricsPeriodMs);
-                  int cpuVcores = ptInfo.getCpuVcores();
-                  final int vmemLimit = (int) (ptInfo.getVmemLimit() >> 20);
-                  final int pmemLimit = (int) (ptInfo.getPmemLimit() >> 20);
-                  usageMetrics.recordResourceLimit(
-                      vmemLimit, pmemLimit, cpuVcores);
                   usageMetrics.recordProcessId(pId);
                 }
               }
@@ -548,7 +534,7 @@ public class ContainersMonitorImpl extends AbstractService implements
               eventDispatcher.getEventHandler().handle(
                   new ContainerKillEvent(containerId,
                       containerExitStatus, msg));
-              it.remove();
+              trackingContainers.remove(containerId);
               LOG.info("Removed ProcessTree with root " + pId);
             }
           } catch (Exception e) {
@@ -605,6 +591,60 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
+  private void changeContainerResource(
+      ContainerId containerId, Resource resource) {
+    Container container = context.getContainers().get(containerId);
+    // Check container existence
+    if (container == null) {
+      LOG.warn("Container " + containerId.toString() + "does not exist");
+      return;
+    }
+    container.setResource(resource);
+  }
+
+  private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
+    if (!containerMetricsEnabled || monitoringEvent == null) {
+      return;
+    }
+
+    ContainerId containerId = monitoringEvent.getContainerId();
+    ContainerMetrics usageMetrics = ContainerMetrics
+        .forContainer(containerId, containerMetricsPeriodMs);
+
+    int vmemLimitMBs;
+    int pmemLimitMBs;
+    int cpuVcores;
+    switch (monitoringEvent.getType()) {
+    case START_MONITORING_CONTAINER:
+      ContainerStartMonitoringEvent startEvent =
+          (ContainerStartMonitoringEvent) monitoringEvent;
+      usageMetrics.recordStateChangeDurations(
+          startEvent.getLaunchDuration(),
+          startEvent.getLocalizationDuration());
+      cpuVcores = startEvent.getCpuVcores();
+      vmemLimitMBs = (int) (startEvent.getVmemLimit() >> 20);
+      pmemLimitMBs = (int) (startEvent.getPmemLimit() >> 20);
+      usageMetrics.recordResourceLimit(
+          vmemLimitMBs, pmemLimitMBs, cpuVcores);
+      break;
+    case STOP_MONITORING_CONTAINER:
+      usageMetrics.finished();
+      break;
+    case CHANGE_MONITORING_CONTAINER_RESOURCE:
+      ChangeMonitoringContainerResourceEvent changeEvent =
+          (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+      Resource resource = changeEvent.getResource();
+      pmemLimitMBs = resource.getMemory();
+      vmemLimitMBs = (int) (pmemLimitMBs * vmemRatio);
+      cpuVcores = resource.getVirtualCores();
+      usageMetrics.recordResourceLimit(
+          vmemLimitMBs, pmemLimitMBs, cpuVcores);
+      break;
+    default:
+      break;
+    }
+  }
+
   @Override
   public long getVmemAllocatedForContainers() {
     return this.maxVmemAllottedForContainers;
@@ -650,38 +690,53 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   @Override
+  @SuppressWarnings("unchecked")
   public void handle(ContainersMonitorEvent monitoringEvent) {
-
+    ContainerId containerId = monitoringEvent.getContainerId();
     if (!containersMonitorEnabled) {
+      if (monitoringEvent.getType() == ContainersMonitorEventType
+          .CHANGE_MONITORING_CONTAINER_RESOURCE) {
+        // Nothing to enforce. Update container resource immediately.
+        ChangeMonitoringContainerResourceEvent changeEvent =
+            (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+        changeContainerResource(containerId, changeEvent.getResource());
+      }
       return;
     }
 
-    ContainerId containerId = monitoringEvent.getContainerId();
     switch (monitoringEvent.getType()) {
     case START_MONITORING_CONTAINER:
       ContainerStartMonitoringEvent startEvent =
           (ContainerStartMonitoringEvent) monitoringEvent;
-
-      if (containerMetricsEnabled) {
-        ContainerMetrics usageMetrics = ContainerMetrics
-            .forContainer(containerId, containerMetricsPeriodMs);
-        usageMetrics.recordStateChangeDurations(
-            startEvent.getLaunchDuration(),
-            startEvent.getLocalizationDuration());
-      }
-
-      synchronized (this.containersToBeAdded) {
-        ProcessTreeInfo processTreeInfo =
-            new ProcessTreeInfo(containerId, null, null,
-                startEvent.getVmemLimit(), startEvent.getPmemLimit(),
-                startEvent.getCpuVcores());
-        this.containersToBeAdded.put(containerId, processTreeInfo);
-      }
+      LOG.info("Starting resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      trackingContainers.put(containerId,
+          new ProcessTreeInfo(containerId, null, null,
+              startEvent.getVmemLimit(), startEvent.getPmemLimit(),
+              startEvent.getCpuVcores()));
       break;
     case STOP_MONITORING_CONTAINER:
-      synchronized (this.containersToBeRemoved) {
-        this.containersToBeRemoved.add(containerId);
+      LOG.info("Stopping resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      trackingContainers.remove(containerId);
+      break;
+    case CHANGE_MONITORING_CONTAINER_RESOURCE:
+      ChangeMonitoringContainerResourceEvent changeEvent =
+          (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+      ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId);
+      if (processTreeInfo == null) {
+        LOG.warn("Failed to track container "
+            + containerId.toString()
+            + ". It may have already completed.");
+        break;
       }
+      LOG.info("Changing resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L;
+      long vmemLimit = (long) (pmemLimit * vmemRatio);
+      int cpuVcores = changeEvent.getResource().getVirtualCores();
+      processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
+      changeContainerResource(containerId, changeEvent.getResource());
       break;
     default:
       // TODO: Wrong event.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.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/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index 9a05278..75bcdae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -211,6 +211,17 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     super.testIncreaseContainerResourceWithInvalidResource();
   }
 
+  @Override
+  public void testChangeContainerResource() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testChangeContainerResource");
+    super.testChangeContainerResource();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index e2f12ba..2ea9146 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -1046,6 +1046,102 @@ public class TestContainerManager extends BaseContainerManagerTest {
     }
   }
 
+  @Test
+  public void testChangeContainerResource() throws Exception {
+    containerManager.start();
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha =
+        recordFactory.newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, rsrc_alpha);
+    containerLaunchContext.setLocalResources(localResources);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    containerLaunchContext.setCommands(commands);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+                createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                    context.getNodeId(), user,
+                        context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request.
+    Resource targetResource = Resource.newInstance(4096, 2);
+    Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+        context.getNodeId(), user, targetResource,
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest.newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    Assert.assertEquals(
+        1, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+    // Check status
+    List<ContainerId> containerIds = new ArrayList<>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    ContainerStatus containerStatus = containerManager
+        .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+    // Check status immediately as resource increase is blocking
+    assertEquals(targetResource, containerStatus.getCapability());
+    // Simulate a decrease request
+    List<org.apache.hadoop.yarn.api.records.Container> containersToDecrease
+        = new ArrayList<>();
+    targetResource = Resource.newInstance(2048, 2);
+    org.apache.hadoop.yarn.api.records.Container decreasedContainer =
+        org.apache.hadoop.yarn.api.records.Container
+            .newInstance(cId, null, null, targetResource, null, null);
+    containersToDecrease.add(decreasedContainer);
+    containerManager.handle(
+        new CMgrDecreaseContainersResourceEvent(containersToDecrease));
+    // Check status with retry
+    containerStatus = containerManager
+        .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+    int retry = 0;
+    while (!targetResource.equals(containerStatus.getCapability()) &&
+        (retry++ < 5)) {
+      Thread.sleep(200);
+      containerStatus = containerManager.getContainerStatuses(gcsRequest)
+          .getContainerStatuses().get(0);
+    }
+    assertEquals(targetResource, containerStatus.getCapability());
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.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/monitor/MockResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java
new file mode 100644
index 0000000..4a18a8c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.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.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+
+public class MockResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+
+  @Override
+  public long getVirtualMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getPhysicalMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public int getNumProcessors() {
+    return 0;
+  }
+
+  @Override
+  public int getNumCores() {
+    return  0;
+  }
+
+  @Override
+  public long getCpuFrequency() {
+    return 0;
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public float getCpuUsage() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.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/monitor/MockResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
new file mode 100644
index 0000000..c5aaa77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+public class MockResourceCalculatorProcessTree extends ResourceCalculatorProcessTree {
+
+  private long rssMemorySize = 0;
+
+  public MockResourceCalculatorProcessTree(String root) {
+    super(root);
+  }
+
+  @Override
+  public void updateProcessTree() {
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    return "";
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    return true;
+  }
+
+  public void setRssMemorySize(long rssMemorySize) {
+    this.rssMemorySize = rssMemorySize;
+  }
+
+  public long getRssMemorySize() {
+    return this.rssMemorySize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d70f938/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.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/monitor/TestContainersMonitorResourceChange.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
new file mode 100644
index 0000000..d7f89fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+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.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ProcessTreeInfo;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerLivenessContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+public class TestContainersMonitorResourceChange {
+
+  private ContainersMonitorImpl containersMonitor;
+  private MockExecutor executor;
+  private Configuration conf;
+  private AsyncDispatcher dispatcher;
+  private Context context;
+  private MockContainerEventHandler containerEventHandler;
+
+  private static class MockExecutor extends ContainerExecutor {
+    @Override
+    public void init() throws IOException {
+    }
+    @Override
+    public void startLocalizer(LocalizerStartContext ctx)
+        throws IOException, InterruptedException {
+    }
+    @Override
+    public int launchContainer(ContainerStartContext ctx) throws
+        IOException {
+      return 0;
+    }
+    @Override
+    public boolean signalContainer(ContainerSignalContext ctx)
+        throws IOException {
+      return true;
+    }
+    @Override
+    public void deleteAsUser(DeletionAsUserContext ctx)
+        throws IOException, InterruptedException {
+    }
+    @Override
+    public String getProcessId(ContainerId containerId) {
+      return String.valueOf(containerId.getContainerId());
+    }
+    @Override
+    public boolean isContainerAlive(ContainerLivenessContext ctx)
+        throws IOException {
+      return true;
+    }
+  }
+
+  private static class MockContainerEventHandler implements
+      EventHandler<ContainerEvent> {
+    final private Set<ContainerId> killedContainer
+        = new HashSet<>();
+    @Override
+    public void handle(ContainerEvent event) {
+      if (event.getType() == ContainerEventType.KILL_CONTAINER) {
+        synchronized (killedContainer) {
+          killedContainer.add(event.getContainerID());
+        }
+      }
+    }
+    public boolean isContainerKilled(ContainerId containerId) {
+      synchronized (killedContainer) {
+        return killedContainer.contains(containerId);
+      }
+    }
+  }
+
+  @Before
+  public void setup() {
+    executor = new MockExecutor();
+    dispatcher = new AsyncDispatcher();
+    context = Mockito.mock(Context.class);
+    Mockito.doReturn(new ConcurrentSkipListMap<ContainerId, Container>())
+        .when(context).getContainers();
+    conf = new Configuration();
+    conf.set(
+        YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
+        MockResourceCalculatorPlugin.class.getCanonicalName());
+    conf.set(
+        YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
+        MockResourceCalculatorProcessTree.class.getCanonicalName());
+    dispatcher.init(conf);
+    dispatcher.start();
+    containerEventHandler = new MockContainerEventHandler();
+    dispatcher.register(ContainerEventType.class, containerEventHandler);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (containersMonitor != null) {
+      containersMonitor.stop();
+    }
+    if (dispatcher != null) {
+      dispatcher.stop();
+    }
+  }
+
+  @Test
+  public void testContainersResourceChange() throws Exception {
+    // set container monitor interval to be 20ms
+    conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    containersMonitor.init(conf);
+    containersMonitor.start();
+    // create container 1
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(1), 2100L, 1000L, 1, 0, 0));
+    // verify that this container is properly tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(1)));
+    assertEquals(1000L, getProcessTreeInfo(getContainerId(1))
+        .getPmemLimit());
+    assertEquals(2100L, getProcessTreeInfo(getContainerId(1))
+        .getVmemLimit());
+    // sleep longer than the monitor interval to make sure resource
+    // enforcement has started
+    Thread.sleep(200);
+    // increase pmem usage, the container should be killed
+    MockResourceCalculatorProcessTree mockTree =
+        (MockResourceCalculatorProcessTree) getProcessTreeInfo(
+            getContainerId(1)).getProcessTree();
+    mockTree.setRssMemorySize(2500L);
+    // verify that this container is killed
+    Thread.sleep(200);
+    assertTrue(containerEventHandler
+        .isContainerKilled(getContainerId(1)));
+    // create container 2
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(2), 2202009L, 1048576L, 1, 0, 0));
+    // verify that this container is properly tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(2)));
+    assertEquals(1048576L, getProcessTreeInfo(getContainerId(2))
+        .getPmemLimit());
+    assertEquals(2202009L, getProcessTreeInfo(getContainerId(2))
+        .getVmemLimit());
+    // trigger a change resource event, check limit after change
+    containersMonitor.handle(new ChangeMonitoringContainerResourceEvent(
+        getContainerId(2), Resource.newInstance(2, 1)));
+    assertEquals(2097152L, getProcessTreeInfo(getContainerId(2))
+        .getPmemLimit());
+    assertEquals(4404019L, getProcessTreeInfo(getContainerId(2))
+        .getVmemLimit());
+    // sleep longer than the monitor interval to make sure resource
+    // enforcement has started
+    Thread.sleep(200);
+    // increase pmem usage, the container should NOT be killed
+    mockTree =
+        (MockResourceCalculatorProcessTree) getProcessTreeInfo(
+            getContainerId(2)).getProcessTree();
+    mockTree.setRssMemorySize(2000000L);
+    // verify that this container is not killed
+    Thread.sleep(200);
+    assertFalse(containerEventHandler
+        .isContainerKilled(getContainerId(2)));
+    containersMonitor.stop();
+  }
+
+  @Test
+  public void testContainersResourceChangeIsTriggeredImmediately()
+      throws Exception {
+    // set container monitor interval to be 20s
+    conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20000L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    containersMonitor.init(conf);
+    containersMonitor.start();
+    // sleep 1 second to make sure the container monitor thread is
+    // now waiting for the next monitor cycle
+    Thread.sleep(1000);
+    // create a container with id 3
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(3), 2202009L, 1048576L, 1, 0, 0));
+    // Verify that this container has been tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(3)));
+    // trigger a change resource event, check limit after change
+    containersMonitor.handle(new ChangeMonitoringContainerResourceEvent(
+        getContainerId(3), Resource.newInstance(2, 1)));
+    // verify that this container has been properly tracked with the
+    // correct size
+    assertEquals(2097152L, getProcessTreeInfo(getContainerId(3))
+        .getPmemLimit());
+    assertEquals(4404019L, getProcessTreeInfo(getContainerId(3))
+        .getVmemLimit());
+    containersMonitor.stop();
+  }
+
+  private ContainersMonitorImpl createContainersMonitor(
+      ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
+      Context context) {
+    return new ContainersMonitorImpl(containerExecutor, dispatcher, context);
+  }
+
+  private ContainerId getContainerId(int id) {
+    return ContainerId.newContainerId(ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(123456L, 1), 1), id);
+  }
+
+  private ProcessTreeInfo getProcessTreeInfo(ContainerId id) {
+    return containersMonitor.trackingContainers.get(id);
+  }
+}


[13/50] [abbrv] hadoop git commit: YARN-2801. Add documentation for node labels feature. Contributed by Wangda Tan and Naganarasimha G R.

Posted by wa...@apache.org.
YARN-2801. Add documentation for node labels feature. Contributed by Wangda Tan and Naganarasimha G R.


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

Branch: refs/heads/YARN-1197
Commit: faa38e1aa49907254bf981662a8aeb5dc52e75e3
Parents: bf669b6
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Sep 1 17:54:14 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Sep 1 17:54:49 2015 +0900

----------------------------------------------------------------------
 hadoop-project/src/site/site.xml                |   1 +
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../src/site/markdown/NodeLabel.md              | 140 +++++++++++++++++++
 3 files changed, 144 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/faa38e1a/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index ee0dfcd..272ea7b 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -121,6 +121,7 @@
       <item name="Fair Scheduler" href="hadoop-yarn/hadoop-yarn-site/FairScheduler.html"/>
       <item name="ResourceManager Restart" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRestart.html"/>
       <item name="ResourceManager HA" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerHA.html"/>
+      <item name="Node Labels" href="hadoop-yarn/hadoop-yarn-site/NodeLabel.html"/>
       <item name="Web Application Proxy" href="hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html"/>
       <item name="Writing YARN Applications" href="hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/faa38e1a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 999654d..e2b1307 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -827,6 +827,9 @@ Release 2.7.2 - UNRELEASED
     YARN-4092. Fixed UI redirection to print useful messages when both RMs are
     in standby mode. (Xuan Gong via jianhe)
 
+    YARN-2801. Add documentation for node labels feature. (Wangda Tan and Naganarasimha 
+    G R  via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/faa38e1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
new file mode 100644
index 0000000..87019cd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md
@@ -0,0 +1,140 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+YARN Node Labels
+===============
+
+# Overview
+Node label is a way to group nodes with similar characteristics and applications can specify where to run.
+
+Now we only support node partition, which is:
+
+* One node can have only one node partition, so a cluster is partitioned to several disjoint sub-clusters by node partitions. By default, nodes belong to DEFAULT partition (partition="")
+* User need to configure how much resources of each partition can be used by different queues. For more detail, please refer next section.
+* There are two kinds of node partitions:
+    * Exclusive: containers will be allocated to nodes with exactly match node partition. (e.g. asking partition=“x” will be allocated to node with partition=“x”, asking DEFAULT partition will be allocated to DEFAULT partition nodes).
+    * Non-exclusive: if a partition is non-exclusive, it shares idle resource to container requesting DEFAULT partition.
+
+User can specify set of node labels which can be accessed by each queue, one application can only use subset of node labels that can be accessed by the queue which contains the application.
+
+# Features
+The ```Node Labels``` supports the following features for now:
+
+* Partition cluster - each node can be assigned one label, so the cluster will be divided to several smaller disjoint partitions.
+* ACL of node-labels on queues - user can set accessible node labels on each queue so only some nodes can only be accessed by specific queues.
+* Specify percentage of resource of a partition which can be accessed by a queue - user can set percentage like: queue A can access 30% of resources on nodes with label=hbase. Such percentage setting will be consistent with existing resource manager
+* Specify required Node Label in resource request, it will only be allocated when node has the same label. If no node label requirement specified, such Resource Request will only be allocated on nodes belong to DEFAULT partition.
+* Operability
+    * Node labels and node labels mapping can be recovered across RM restart
+    * Update node labels - admin can update labels on nodes and labels on queues
+      when RM is running
+
+# Configuration
+## Setting up ```ResourceManager``` to enable ```Node Labels```:
+
+Setup following properties in ```yarn-site.xml```
+
+Property  | Value
+--- | ----
+yarn.node-labels.fs-store.root-dir  | hdfs://namenode:port/path/to/store/node-labels/
+yarn.node-labels.enabled | true
+
+Notes:
+
+* Make sure ```yarn.node-labels.fs-store.root-dir``` is created and ```ResourceManager``` has permission to access it. (Typically from “yarn” user)
+* If user want to store node label to local file system of RM (instead of HDFS), paths like `file:///home/yarn/node-label` can be used
+
+### Add/modify node labels list and node-to-labels mapping to YARN
+* Add cluster node labels list:
+    * Executing ```yarn rmadmin -addToClusterNodeLabels "label_1(exclusive=true/false),label_2(exclusive=true/false)"``` to add node label.
+    * If user don’t specify “(exclusive=…)”, execlusive will be ```true``` by default.
+    * Run ```yarn cluster --list-node-labels``` to check added node labels are visible in the cluster.
+
+* Add labels to nodes
+    * Executing ```yarn rmadmin -replaceLabelsOnNode “node1[:port]=label1 node2=label2”```. Added label1 to node1, label2 to node2. If user don’t specify port, it added the label to all ```NodeManagers``` running on the node.
+
+## Configuration of Schedulers for node labels
+### Capacity Scheduler Configuration
+Property  | Value
+----- | ------
+yarn.scheduler.capacity.`<queue-path>`.capacity | Set the percentage of the queue can access to nodes belong to DEFAULT partition. The sum of DEFAULT capacities for direct children under each parent, must be equal to 100.
+yarn.scheduler.capacity.`<queue-path>`.accessible-node-labels | Admin need specify labels can be accessible by each queue, split by comma, like “hbase,storm” means queue can access label hbase and storm. All queues can access to nodes without label, user don’t have to specify that. If user don’t specify this field, it will inherit from its parent. If user want to explicitly specify a queue can only access nodes without labels, just put a space as the value.
+yarn.scheduler.capacity.`<queue-path>`.accessible-node-labels.`<label>`.capacity | Set the percentage of the queue can access to nodes belong to `<label>` partition . The sum of `<label>` capacities for direct children under each parent, must be equal to 100. By default, it's 0.
+yarn.scheduler.capacity.`<queue-path>`.accessible-node-labels.`<label>`.maximum-capacity | Similar to yarn.scheduler.capacity.`<queue-path>`.maximum-capacity, it is for maximum-capacity for labels of each queue. By default, it's 100.
+yarn.scheduler.capacity.`<queue-path>`.default-node-label-expression | Value like “hbase”, which means: if applications submitted to the queue without specifying node label in their resource requests, it will use "hbase" as default-node-label-expression. By default, this is empty, so application will get containers from nodes without label.
+
+**An example of node label configuration**:
+
+Assume we have a queue structure
+
+```
+                root
+            /     |    \
+     engineer    sales  marketing
+```
+
+We have 5 nodes (hostname=h1..h5) in the cluster, each of them has 24G memory, 24 vcores. 1 among the 5 nodes has GPU (assume it’s h5). So admin added GPU label to h5.
+
+Assume user have a Capacity Scheduler configuration like: (key=value is used here for readability)
+
+```
+yarn.scheduler.capacity.root.queues=engineering,marketing,sales
+yarn.scheduler.capacity.root.engineering.capacity=33
+yarn.scheduler.capacity.root.marketing.capacity=34
+yarn.scheduler.capacity.root.sales.capacity=33
+
+yarn.scheduler.capacity.root.engineering.accessible-node-labels=GPU
+yarn.scheduler.capacity.root.marketing.accessible-node-labels=GPU
+
+yarn.scheduler.capacity.root.engineering.accessible-node-labels.GPU.capacity=50
+yarn.scheduler.capacity.root.marketing.accessible-node-labels.GPU.capacity=50
+
+yarn.scheduler.capacity.root.engineering.default-node-label-expression=GPU
+```
+
+You can see root.engineering/marketing/sales.capacity=33, so each of them can has guaranteed resource equals to 1/3 of resource **without partition**. So each of them can use 1/3 resource of h1..h4, which is 24 * 4 * (1/3) = (32G mem, 32 v-cores).
+
+And only engineering/marketing queue has permission to access GPU partition (see root.`<queue-name>`.accessible-node-labels).
+
+Each of engineering/marketing queue has guaranteed resource equals to 1/2 of resource **with partition=GPU**. So each of them can use 1/2 resource of h5, which is 24 * 0.5 = (12G mem, 12 v-cores).
+
+Notes:
+
+* After finishing configuration of CapacityScheduler, execute ```yarn rmadmin -refreshQueues``` to apply changes
+* Go to scheduler page of RM Web UI to check if you have successfully set configuration.
+
+# Specifying node label for application
+Applications can use following Java APIs to specify node label to request
+
+* `ApplicationSubmissionContext.setNodeLabelExpression(..)` to set node label expression for all containers of the application.
+* `ResourceRequest.setNodeLabelExpression(..)` to set node label expression for individual resource requests. This can overwrite node label expression set in ApplicationSubmissionContext
+* Specify `setAMContainerResourceRequest.setNodeLabelExpression` in `ApplicationSubmissionContext` to indicate expected node label for application master container.
+
+# Monitoring
+
+## Monitoring through web UI
+Following label-related fields can be seen on web UI:
+
+* Nodes page: http://RM-Address:port/cluster/nodes, you can get labels on each node
+* Node labels page: http://RM-Address:port/cluster/nodelabels, you can get type (exclusive/non-exclusive), number of active node managers, total resource of each partition
+* Scheduler page: http://RM-Address:port/cluster/scheduler, you can get label-related settings of each queue, and resource usage of queue partitions.
+
+## Monitoring through commandline
+
+* Use `yarn cluster --list-node-labels` to get labels in the cluster
+* Use `yarn node -status <NodeId>` to get node status including labels on a given node
+
+# Useful links
+* [YARN Capacity Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html), if you need more understanding about how to configure Capacity Scheduler
+* Write YARN application using node labels, you can see following two links as examples: [YARN distributed shell](https://issues.apache.org/jira/browse/YARN-2502), [Hadoop MapReduce](https://issues.apache.org/jira/browse/MAPREDUCE-6304)


[07/50] [abbrv] hadoop git commit: Move YARN-4092 to 2.7.2

Posted by wa...@apache.org.
Move YARN-4092 to 2.7.2


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

Branch: refs/heads/YARN-1197
Commit: 4eaa7fd3eae4412ac0b964c617b1bbb17a39d8be
Parents: a3fd2cc
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 31 17:43:36 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Aug 31 17:43:36 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4eaa7fd3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 19c1082..80cf793 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -422,9 +422,6 @@ Release 2.8.0 - UNRELEASED
     YARN-1556. NPE getting application report with a null appId. (Weiwei Yang via 
     junping_du)
 
-    YARN-4092. Fixed UI redirection to print useful messages when both RMs are
-    in standby mode. (Xuan Gong via jianhe)
-
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not
@@ -824,6 +821,8 @@ Release 2.7.2 - UNRELEASED
     YARN-3978. Configurably turn off the saving of container info in Generic AHS
     (Eric Payne via jeagles)
 
+    YARN-4092. Fixed UI redirection to print useful messages when both RMs are
+    in standby mode. (Xuan Gong via jianhe)
 
   OPTIMIZATIONS
 


[42/50] [abbrv] hadoop git commit: HDFS-9012. Move o.a.h.hdfs.protocol.datatransfer.PipelineAck class to hadoop-hdfs-client module. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
HDFS-9012. Move o.a.h.hdfs.protocol.datatransfer.PipelineAck class to hadoop-hdfs-client module. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: d16c4eee186492608ffeb1c2e83f437000cc64f6
Parents: 6eaca2e
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Sep 4 10:41:09 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Sep 4 10:41:09 2015 -0700

----------------------------------------------------------------------
 .../hdfs/protocol/datatransfer/PipelineAck.java | 243 ++++++++++++++++
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |  71 +++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/protocol/datatransfer/PipelineAck.java | 274 -------------------
 .../hdfs/server/datanode/BlockReceiver.java     |   2 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  38 +++
 .../apache/hadoop/hdfs/util/LongBitFormat.java  |  71 -----
 7 files changed, 356 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
new file mode 100644
index 0000000..3836606
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
@@ -0,0 +1,243 @@
+/**
+ * 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.protocol.datatransfer;
+
+import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.hdfs.util.LongBitFormat;
+
+/** Pipeline Acknowledgment **/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PipelineAck {
+  PipelineAckProto proto;
+  public final static long UNKOWN_SEQNO = -2;
+  final static int OOB_START = Status.OOB_RESTART_VALUE; // the first OOB type
+  final static int OOB_END = Status.OOB_RESERVED3_VALUE; // the last OOB type
+
+  public enum ECN {
+    DISABLED(0),
+    SUPPORTED(1),
+    SUPPORTED2(2),
+    CONGESTED(3);
+
+    private final int value;
+    private static final ECN[] VALUES = values();
+    static ECN valueOf(int value) {
+      return VALUES[value];
+    }
+
+    ECN(int value) {
+      this.value = value;
+    }
+
+    public int getValue() {
+      return value;
+    }
+  }
+
+  private enum StatusFormat {
+    STATUS(null, 4),
+    RESERVED(STATUS.BITS, 1),
+    ECN_BITS(RESERVED.BITS, 2);
+
+    private final LongBitFormat BITS;
+
+    StatusFormat(LongBitFormat prev, int bits) {
+      BITS = new LongBitFormat(name(), prev, bits, 0);
+    }
+
+    static Status getStatus(int header) {
+      return Status.valueOf((int) STATUS.BITS.retrieve(header));
+    }
+
+    static ECN getECN(int header) {
+      return ECN.valueOf((int) ECN_BITS.BITS.retrieve(header));
+    }
+
+    public static int setStatus(int old, Status status) {
+      return (int) STATUS.BITS.combine(status.getNumber(), old);
+    }
+
+    public static int setECN(int old, ECN ecn) {
+      return (int) ECN_BITS.BITS.combine(ecn.getValue(), old);
+    }
+  }
+
+  /** default constructor **/
+  public PipelineAck() {
+  }
+  
+  /**
+   * Constructor assuming no next DN in pipeline
+   * @param seqno sequence number
+   * @param replies an array of replies
+   */
+  public PipelineAck(long seqno, int[] replies) {
+    this(seqno, replies, 0L);
+  }
+
+  /**
+   * Constructor
+   * @param seqno sequence number
+   * @param replies an array of replies
+   * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline
+   */
+  public PipelineAck(long seqno, int[] replies,
+                     long downstreamAckTimeNanos) {
+    ArrayList<Status> statusList = Lists.newArrayList();
+    ArrayList<Integer> flagList = Lists.newArrayList();
+    for (int r : replies) {
+      statusList.add(StatusFormat.getStatus(r));
+      flagList.add(r);
+    }
+    proto = PipelineAckProto.newBuilder()
+      .setSeqno(seqno)
+      .addAllReply(statusList)
+      .addAllFlag(flagList)
+      .setDownstreamAckTimeNanos(downstreamAckTimeNanos)
+      .build();
+  }
+  
+  /**
+   * Get the sequence number
+   * @return the sequence number
+   */
+  public long getSeqno() {
+    return proto.getSeqno();
+  }
+  
+  /**
+   * Get the number of replies
+   * @return the number of replies
+   */
+  public short getNumOfReplies() {
+    return (short)proto.getReplyCount();
+  }
+  
+  /**
+   * get the header flag of ith reply
+   */
+  public int getHeaderFlag(int i) {
+    if (proto.getFlagCount() > 0) {
+      return proto.getFlag(i);
+    } else {
+      return combineHeader(ECN.DISABLED, proto.getReply(i));
+    }
+  }
+
+  public int getFlag(int i) {
+    return proto.getFlag(i);
+  }
+
+  /**
+   * Get the time elapsed for downstream ack RTT in nanoseconds
+   * @return time elapsed for downstream ack in nanoseconds, 0 if no next DN in pipeline
+   */
+  public long getDownstreamAckTimeNanos() {
+    return proto.getDownstreamAckTimeNanos();
+  }
+
+  /**
+   * Check if this ack contains error status
+   * @return true if all statuses are SUCCESS
+   */
+  public boolean isSuccess() {
+    for (Status s : proto.getReplyList()) {
+      if (s != Status.SUCCESS) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Returns the OOB status if this ack contains one. 
+   * @return null if it is not an OOB ack.
+   */
+  public Status getOOBStatus() {
+    // Normal data transfer acks will have a valid sequence number, so
+    // this will return right away in most cases.
+    if (getSeqno() != UNKOWN_SEQNO) {
+      return null;
+    }
+    for (Status s : proto.getReplyList()) {
+      // The following check is valid because protobuf guarantees to
+      // preserve the ordering of enum elements.
+      if (s.getNumber() >= OOB_START && s.getNumber() <= OOB_END) {
+        return s;
+      }
+    }
+    return null;
+  }
+
+  /** Get the Restart OOB ack status */
+  public static Status getRestartOOBStatus() {
+    return Status.OOB_RESTART;
+  }
+
+  /** return true if it is the restart OOB status code  */
+  public static boolean isRestartOOBStatus(Status st) {
+    return st.equals(Status.OOB_RESTART);
+  }
+
+  /**** Writable interface ****/
+  public void readFields(InputStream in) throws IOException {
+    proto = PipelineAckProto.parseFrom(vintPrefixed(in));
+  }
+
+  public void write(OutputStream out) throws IOException {
+    proto.writeDelimitedTo(out);
+  }
+  
+  @Override //Object
+  public String toString() {
+    return TextFormat.shortDebugString(proto);
+  }
+
+  public static Status getStatusFromHeader(int header) {
+    return StatusFormat.getStatus(header);
+  }
+
+  public static ECN getECNFromHeader(int header) {
+    return StatusFormat.getECN(header);
+  }
+
+  public static int setStatusForHeader(int old, Status status) {
+    return StatusFormat.setStatus(old, status);
+  }
+
+  public static int combineHeader(ECN ecn, Status status) {
+    int header = 0;
+    header = StatusFormat.setStatus(header, status);
+    header = StatusFormat.setECN(header, ecn);
+    return header;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
new file mode 100644
index 0000000..9399d84
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.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.util;
+
+import java.io.Serializable;
+
+
+/**
+ * Bit format in a long.
+ */
+public class LongBitFormat implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final String NAME;
+  /** Bit offset */
+  private final int OFFSET;
+  /** Bit length */
+  private final int LENGTH;
+  /** Minimum value */
+  private final long MIN;
+  /** Maximum value */
+  private final long MAX;
+  /** Bit mask */
+  private final long MASK;
+
+  public LongBitFormat(String name, LongBitFormat previous, int length, long min) {
+    NAME = name;
+    OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH;
+    LENGTH = length;
+    MIN = min;
+    MAX = ((-1L) >>> (64 - LENGTH));
+    MASK = MAX << OFFSET;
+  }
+
+  /** Retrieve the value from the record. */
+  public long retrieve(long record) {
+    return (record & MASK) >>> OFFSET;
+  }
+
+  /** Combine the value to the record. */
+  public long combine(long value, long record) {
+    if (value < MIN) {
+      throw new IllegalArgumentException(
+          "Illagal value: " + NAME + " = " + value + " < MIN = " + MIN);
+    }
+    if (value > MAX) {
+      throw new IllegalArgumentException(
+          "Illagal value: " + NAME + " = " + value + " > MAX = " + MAX);
+    }
+    return (record & ~MASK) | (value << OFFSET);
+  }
+  
+  public long getMin() {
+    return MIN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b9b89aa..e67c9d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -888,6 +888,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9021. Use a yellow elephant rather than a blue one in diagram. (wang)
 
+    HDFS-9012. Move o.a.h.hdfs.protocol.datatransfer.PipelineAck class to
+    hadoop-hdfs-client module. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
deleted file mode 100644
index 44f38c6..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
+++ /dev/null
@@ -1,274 +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.hdfs.protocol.datatransfer;
-
-import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_DEFAULT;
-
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import com.google.protobuf.TextFormat;
-import org.apache.hadoop.hdfs.util.LongBitFormat;
-
-/** Pipeline Acknowledgment **/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class PipelineAck {
-  PipelineAckProto proto;
-  public final static long UNKOWN_SEQNO = -2;
-  final static int OOB_START = Status.OOB_RESTART_VALUE; // the first OOB type
-  final static int OOB_END = Status.OOB_RESERVED3_VALUE; // the last OOB type
-  final static int NUM_OOB_TYPES = OOB_END - OOB_START + 1;
-  // place holder for timeout value of each OOB type
-  final static long[] OOB_TIMEOUT;
-
-  public enum ECN {
-    DISABLED(0),
-    SUPPORTED(1),
-    SUPPORTED2(2),
-    CONGESTED(3);
-
-    private final int value;
-    private static final ECN[] VALUES = values();
-    static ECN valueOf(int value) {
-      return VALUES[value];
-    }
-
-    ECN(int value) {
-      this.value = value;
-    }
-
-    public int getValue() {
-      return value;
-    }
-  }
-
-  private enum StatusFormat {
-    STATUS(null, 4),
-    RESERVED(STATUS.BITS, 1),
-    ECN_BITS(RESERVED.BITS, 2);
-
-    private final LongBitFormat BITS;
-
-    StatusFormat(LongBitFormat prev, int bits) {
-      BITS = new LongBitFormat(name(), prev, bits, 0);
-    }
-
-    static Status getStatus(int header) {
-      return Status.valueOf((int) STATUS.BITS.retrieve(header));
-    }
-
-    static ECN getECN(int header) {
-      return ECN.valueOf((int) ECN_BITS.BITS.retrieve(header));
-    }
-
-    public static int setStatus(int old, Status status) {
-      return (int) STATUS.BITS.combine(status.getNumber(), old);
-    }
-
-    public static int setECN(int old, ECN ecn) {
-      return (int) ECN_BITS.BITS.combine(ecn.getValue(), old);
-    }
-  }
-
-  static {
-    OOB_TIMEOUT = new long[NUM_OOB_TYPES];
-    HdfsConfiguration conf = new HdfsConfiguration();
-    String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
-        DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
-    for (int i = 0; i < NUM_OOB_TYPES; i++) {
-      OOB_TIMEOUT[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
-    }
-  }
-
-  /** default constructor **/
-  public PipelineAck() {
-  }
-  
-  /**
-   * Constructor assuming no next DN in pipeline
-   * @param seqno sequence number
-   * @param replies an array of replies
-   */
-  public PipelineAck(long seqno, int[] replies) {
-    this(seqno, replies, 0L);
-  }
-
-  /**
-   * Constructor
-   * @param seqno sequence number
-   * @param replies an array of replies
-   * @param downstreamAckTimeNanos ack RTT in nanoseconds, 0 if no next DN in pipeline
-   */
-  public PipelineAck(long seqno, int[] replies,
-                     long downstreamAckTimeNanos) {
-    ArrayList<Status> statusList = Lists.newArrayList();
-    ArrayList<Integer> flagList = Lists.newArrayList();
-    for (int r : replies) {
-      statusList.add(StatusFormat.getStatus(r));
-      flagList.add(r);
-    }
-    proto = PipelineAckProto.newBuilder()
-      .setSeqno(seqno)
-      .addAllReply(statusList)
-      .addAllFlag(flagList)
-      .setDownstreamAckTimeNanos(downstreamAckTimeNanos)
-      .build();
-  }
-  
-  /**
-   * Get the sequence number
-   * @return the sequence number
-   */
-  public long getSeqno() {
-    return proto.getSeqno();
-  }
-  
-  /**
-   * Get the number of replies
-   * @return the number of replies
-   */
-  public short getNumOfReplies() {
-    return (short)proto.getReplyCount();
-  }
-  
-  /**
-   * get the header flag of ith reply
-   */
-  public int getHeaderFlag(int i) {
-    if (proto.getFlagCount() > 0) {
-      return proto.getFlag(i);
-    } else {
-      return combineHeader(ECN.DISABLED, proto.getReply(i));
-    }
-  }
-
-  public int getFlag(int i) {
-    return proto.getFlag(i);
-  }
-
-  /**
-   * Get the time elapsed for downstream ack RTT in nanoseconds
-   * @return time elapsed for downstream ack in nanoseconds, 0 if no next DN in pipeline
-   */
-  public long getDownstreamAckTimeNanos() {
-    return proto.getDownstreamAckTimeNanos();
-  }
-
-  /**
-   * Check if this ack contains error status
-   * @return true if all statuses are SUCCESS
-   */
-  public boolean isSuccess() {
-    for (Status s : proto.getReplyList()) {
-      if (s != Status.SUCCESS) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Returns the OOB status if this ack contains one. 
-   * @return null if it is not an OOB ack.
-   */
-  public Status getOOBStatus() {
-    // Normal data transfer acks will have a valid sequence number, so
-    // this will return right away in most cases.
-    if (getSeqno() != UNKOWN_SEQNO) {
-      return null;
-    }
-    for (Status s : proto.getReplyList()) {
-      // The following check is valid because protobuf guarantees to
-      // preserve the ordering of enum elements.
-      if (s.getNumber() >= OOB_START && s.getNumber() <= OOB_END) {
-        return s;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Get the timeout to be used for transmitting the OOB type
-   * @return the timeout in milliseconds
-   */
-  public static long getOOBTimeout(Status status) throws IOException {
-    int index = status.getNumber() - OOB_START;
-    if (index >= 0 && index < NUM_OOB_TYPES) {
-      return OOB_TIMEOUT[index];
-    } 
-    // Not an OOB.
-    throw new IOException("Not an OOB status: " + status);
-  }
-
-  /** Get the Restart OOB ack status */
-  public static Status getRestartOOBStatus() {
-    return Status.OOB_RESTART;
-  }
-
-  /** return true if it is the restart OOB status code  */
-  public static boolean isRestartOOBStatus(Status st) {
-    return st.equals(Status.OOB_RESTART);
-  }
-
-  /**** Writable interface ****/
-  public void readFields(InputStream in) throws IOException {
-    proto = PipelineAckProto.parseFrom(vintPrefixed(in));
-  }
-
-  public void write(OutputStream out) throws IOException {
-    proto.writeDelimitedTo(out);
-  }
-  
-  @Override //Object
-  public String toString() {
-    return TextFormat.shortDebugString(proto);
-  }
-
-  public static Status getStatusFromHeader(int header) {
-    return StatusFormat.getStatus(header);
-  }
-
-  public static ECN getECNFromHeader(int header) {
-    return StatusFormat.getECN(header);
-  }
-
-  public static int setStatusForHeader(int old, Status status) {
-    return StatusFormat.setStatus(old, status);
-  }
-
-  public static int combineHeader(ECN ecn, Status status) {
-    int header = 0;
-    header = StatusFormat.setStatus(header, status);
-    header = StatusFormat.setECN(header, ecn);
-    return header;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 1cb308f..bc5396f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -1153,7 +1153,7 @@ class BlockReceiver implements Closeable {
 
       synchronized(this) {
         if (sending) {
-          wait(PipelineAck.getOOBTimeout(ackStatus));
+          wait(datanode.getOOBTimeout(ackStatus));
           // Didn't get my turn in time. Give up.
           if (sending) {
             throw new IOException("Could not send OOB reponse in time: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/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 42cbd96..e0adc6d 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
@@ -40,6 +40,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT;
@@ -359,6 +361,8 @@ public class DataNode extends ReconfigurableBase
       .availableProcessors();
   private static final double CONGESTION_RATIO = 1.5;
 
+  private long[] oobTimeouts; /** timeout value of each OOB type */
+
   /**
    * Creates a dummy DataNode for testing purpose.
    */
@@ -373,6 +377,7 @@ public class DataNode extends ReconfigurableBase
     this.connectToDnViaHostname = false;
     this.blockScanner = new BlockScanner(this, conf);
     this.pipelineSupportECN = false;
+    initOOBTimeout();
   }
 
   /**
@@ -446,6 +451,8 @@ public class DataNode extends ReconfigurableBase
                 return ret;
               }
             });
+
+    initOOBTimeout();
   }
 
   @Override  // ReconfigurableBase
@@ -3226,4 +3233,35 @@ public class DataNode extends ReconfigurableBase
     checkSuperuserPrivilege();
     spanReceiverHost.removeSpanReceiver(id);
   }
+
+  /**
+   * Get timeout value of each OOB type from configuration
+   */
+  private void initOOBTimeout() {
+    final int oobStart = Status.OOB_RESTART_VALUE; // the first OOB type
+    final int oobEnd = Status.OOB_RESERVED3_VALUE; // the last OOB type
+    final int numOobTypes = oobEnd - oobStart + 1;
+    oobTimeouts = new long[numOobTypes];
+
+    final String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
+        DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
+    for (int i = 0; i < numOobTypes; i++) {
+      oobTimeouts[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
+    }
+  }
+
+  /**
+   * Get the timeout to be used for transmitting the OOB type
+   * @return the timeout in milliseconds
+   */
+  public long getOOBTimeout(Status status)
+      throws IOException {
+    if (status.getNumber() < Status.OOB_RESTART_VALUE ||
+        status.getNumber() > Status.OOB_RESERVED3_VALUE) {
+      // Not an OOB.
+      throw new IOException("Not an OOB status: " + status);
+    }
+
+    return oobTimeouts[status.getNumber() - Status.OOB_RESTART_VALUE];
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d16c4eee/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
deleted file mode 100644
index 9399d84..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LongBitFormat.java
+++ /dev/null
@@ -1,71 +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.hdfs.util;
-
-import java.io.Serializable;
-
-
-/**
- * Bit format in a long.
- */
-public class LongBitFormat implements Serializable {
-  private static final long serialVersionUID = 1L;
-
-  private final String NAME;
-  /** Bit offset */
-  private final int OFFSET;
-  /** Bit length */
-  private final int LENGTH;
-  /** Minimum value */
-  private final long MIN;
-  /** Maximum value */
-  private final long MAX;
-  /** Bit mask */
-  private final long MASK;
-
-  public LongBitFormat(String name, LongBitFormat previous, int length, long min) {
-    NAME = name;
-    OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH;
-    LENGTH = length;
-    MIN = min;
-    MAX = ((-1L) >>> (64 - LENGTH));
-    MASK = MAX << OFFSET;
-  }
-
-  /** Retrieve the value from the record. */
-  public long retrieve(long record) {
-    return (record & MASK) >>> OFFSET;
-  }
-
-  /** Combine the value to the record. */
-  public long combine(long value, long record) {
-    if (value < MIN) {
-      throw new IllegalArgumentException(
-          "Illagal value: " + NAME + " = " + value + " < MIN = " + MIN);
-    }
-    if (value > MAX) {
-      throw new IllegalArgumentException(
-          "Illagal value: " + NAME + " = " + value + " > MAX = " + MAX);
-    }
-    return (record & ~MASK) | (value << OFFSET);
-  }
-  
-  public long getMin() {
-    return MIN;
-  }
-}


[49/50] [abbrv] hadoop git commit: YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: cf3d5e544495a40e0cd10b558d1be62e7dae3968
Parents: 5d70f93
Author: Jian He <ji...@apache.org>
Authored: Thu Aug 20 21:04:14 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:04 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/client/TestResourceTrackerOnHA.java    |   2 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |   4 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  76 +++++-
 .../yarn/server/api/records/NodeStatus.java     |  15 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |  75 +++++-
 .../main/proto/yarn_server_common_protos.proto  |   3 +-
 .../yarn_server_common_service_protos.proto     |   1 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |  39 ++-
 .../hadoop/yarn/server/nodemanager/Context.java |   3 +
 .../yarn/server/nodemanager/NodeManager.java    |  10 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |  59 ++++-
 .../containermanager/ContainerManagerImpl.java  | 116 +++++----
 .../nodemanager/TestNodeManagerResync.java      | 258 +++++++++++++++++++
 .../containermanager/TestContainerManager.java  |   2 +-
 15 files changed, 600 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a524bbc..429fa12 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -205,6 +205,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1643. Make ContainersMonitor support changing monitoring size of an
     allocated container. (Meng Ding and Wangda Tan)
 
+    YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to
+    support container resizing. (Meng Ding via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
index 6cdf87f..338198b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
@@ -68,7 +68,7 @@ public class TestResourceTrackerOnHA extends ProtocolHATestBase{
     failoverThread = createAndStartFailoverThread();
     NodeStatus status =
         NodeStatus.newInstance(NodeId.newInstance("localhost", 0), 0, null,
-            null, null, null, null);
+            null, null, null, null, null);
     NodeHeartbeatRequest request2 =
         NodeHeartbeatRequest.newInstance(status, null, null,null);
     resourceTracker.nodeHeartbeat(request2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
index 1498a0c..38fbc82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 
@@ -70,4 +71,7 @@ public interface NodeHeartbeatResponse {
   
   boolean getAreNodeLabelsAcceptedByRM();
   void setAreNodeLabelsAcceptedByRM(boolean areNodeLabelsAcceptedByRM);
+
+  List<Container> getContainersToDecrease();
+  void addAllContainersToDecrease(List<Container> containersToDecrease);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
index e27d8ca..12c5230 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
@@ -27,12 +27,15 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
@@ -58,7 +61,9 @@ public class NodeHeartbeatResponsePBImpl extends
 
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
-  
+
+  private List<Container> containersToDecrease = null;
+
   public NodeHeartbeatResponsePBImpl() {
     builder = NodeHeartbeatResponseProto.newBuilder();
   }
@@ -96,6 +101,9 @@ public class NodeHeartbeatResponsePBImpl extends
     if (this.systemCredentials != null) {
       addSystemCredentialsToProto();
     }
+    if (this.containersToDecrease != null) {
+      addContainersToDecreaseToProto();
+    }
   }
 
   private void addSystemCredentialsToProto() {
@@ -408,6 +416,64 @@ public class NodeHeartbeatResponsePBImpl extends
     builder.addAllApplicationsToCleanup(iterable);
   }
 
+  private void initContainersToDecrease() {
+    if (this.containersToDecrease != null) {
+      return;
+    }
+    NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerProto> list = p.getContainersToDecreaseList();
+    this.containersToDecrease = new ArrayList<>();
+
+    for (ContainerProto c : list) {
+      this.containersToDecrease.add(convertFromProtoFormat(c));
+    }
+  }
+
+  @Override
+  public List<Container> getContainersToDecrease() {
+    initContainersToDecrease();
+    return this.containersToDecrease;
+  }
+
+  @Override
+  public void addAllContainersToDecrease(
+      final List<Container> containersToDecrease) {
+    if (containersToDecrease == null) {
+      return;
+    }
+    initContainersToDecrease();
+    this.containersToDecrease.addAll(containersToDecrease);
+  }
+
+  private void addContainersToDecreaseToProto() {
+    maybeInitBuilder();
+    builder.clearContainersToDecrease();
+    if (this.containersToDecrease == null) {
+      return;
+    }
+    Iterable<ContainerProto> iterable = new
+        Iterable<ContainerProto>() {
+      @Override
+      public Iterator<ContainerProto> iterator() {
+        return new Iterator<ContainerProto>() {
+          private Iterator<Container> iter = containersToDecrease.iterator();
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          @Override
+          public ContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllContainersToDecrease(iterable);
+  }
 
   @Override
   public Map<ApplicationId, ByteBuffer> getSystemCredentialsForApps() {
@@ -484,6 +550,14 @@ public class NodeHeartbeatResponsePBImpl extends
     return ((MasterKeyPBImpl) t).getProto();
   }
 
+  private ContainerPBImpl convertFromProtoFormat(ContainerProto p) {
+    return new ContainerPBImpl(p);
+  }
+
+  private ContainerProto convertToProtoFormat(Container t) {
+    return ((ContainerPBImpl) t).getProto();
+  }
+
   @Override
   public boolean getAreNodeLabelsAcceptedByRM() {
     NodeHeartbeatResponseProtoOrBuilder p =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
index 24391bf..a37c399 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.Records;
@@ -48,6 +49,7 @@ public abstract class NodeStatus {
    * @param nodeHealthStatus Health status of the node.
    * @param containersUtilizations Utilization of the containers in this node.
    * @param nodeUtilization Utilization of the node.
+   * @param increasedContainers Containers whose resource has been increased.
    * @return New {@code NodeStatus} with the provided information.
    */
   public static NodeStatus newInstance(NodeId nodeId, int responseId,
@@ -55,7 +57,8 @@ public abstract class NodeStatus {
       List<ApplicationId> keepAliveApplications,
       NodeHealthStatus nodeHealthStatus,
       ResourceUtilization containersUtilization,
-      ResourceUtilization nodeUtilization) {
+      ResourceUtilization nodeUtilization,
+      List<Container> increasedContainers) {
     NodeStatus nodeStatus = Records.newRecord(NodeStatus.class);
     nodeStatus.setResponseId(responseId);
     nodeStatus.setNodeId(nodeId);
@@ -64,6 +67,7 @@ public abstract class NodeStatus {
     nodeStatus.setNodeHealthStatus(nodeHealthStatus);
     nodeStatus.setContainersUtilization(containersUtilization);
     nodeStatus.setNodeUtilization(nodeUtilization);
+    nodeStatus.setIncreasedContainers(increasedContainers);
     return nodeStatus;
   }
 
@@ -108,4 +112,13 @@ public abstract class NodeStatus {
   @Unstable
   public abstract void setNodeUtilization(
       ResourceUtilization nodeUtilization);
+
+  @Public
+  @Unstable
+  public abstract List<Container> getIncreasedContainers();
+
+  @Private
+  @Unstable
+  public abstract void setIncreasedContainers(
+      List<Container> increasedContainers);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
index 2d139fe..c94febe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
@@ -24,13 +24,16 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
@@ -49,7 +52,8 @@ public class NodeStatusPBImpl extends NodeStatus {
   private List<ContainerStatus> containers = null;
   private NodeHealthStatus nodeHealthStatus = null;
   private List<ApplicationId> keepAliveApplications = null;
-  
+  private List<Container> increasedContainers = null;
+
   public NodeStatusPBImpl() {
     builder = NodeStatusProto.newBuilder();
   }
@@ -79,6 +83,9 @@ public class NodeStatusPBImpl extends NodeStatus {
     if (this.keepAliveApplications != null) {
       addKeepAliveApplicationsToProto();
     }
+    if (this.increasedContainers != null) {
+      addIncreasedContainersToProto();
+    }
   }
 
   private synchronized void mergeLocalToProto() {
@@ -165,6 +172,37 @@ public class NodeStatusPBImpl extends NodeStatus {
     builder.addAllKeepAliveApplications(iterable);
   }
 
+  private synchronized void addIncreasedContainersToProto() {
+    maybeInitBuilder();
+    builder.clearIncreasedContainers();
+    if (increasedContainers == null) {
+      return;
+    }
+    Iterable<ContainerProto> iterable = new
+        Iterable<ContainerProto>() {
+      @Override
+      public Iterator<ContainerProto> iterator() {
+        return new Iterator<ContainerProto>() {
+          private Iterator<Container> iter =
+                  increasedContainers.iterator();
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          @Override
+          public ContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllIncreasedContainers(iterable);
+  }
+
   @Override
   public int hashCode() {
     return getProto().hashCode();
@@ -336,6 +374,31 @@ public class NodeStatusPBImpl extends NodeStatus {
         .setNodeUtilization(convertToProtoFormat(nodeUtilization));
   }
 
+  @Override
+  public synchronized List<Container> getIncreasedContainers() {
+    if (increasedContainers != null) {
+      return increasedContainers;
+    }
+    NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerProto> list = p.getIncreasedContainersList();
+    this.increasedContainers = new ArrayList<>();
+    for (ContainerProto c : list) {
+      this.increasedContainers.add(convertFromProtoFormat(c));
+    }
+    return this.increasedContainers;
+  }
+
+  @Override
+  public synchronized void setIncreasedContainers(
+      List<Container> increasedContainers) {
+    maybeInitBuilder();
+    if (increasedContainers == null) {
+      builder.clearIncreasedContainers();
+      return;
+    }
+    this.increasedContainers = increasedContainers;
+  }
+
   private NodeIdProto convertToProtoFormat(NodeId nodeId) {
     return ((NodeIdPBImpl)nodeId).getProto();
   }
@@ -377,4 +440,14 @@ public class NodeStatusPBImpl extends NodeStatus {
       ResourceUtilizationProto p) {
     return new ResourceUtilizationPBImpl(p);
   }
+
+  private ContainerPBImpl convertFromProtoFormat(
+      ContainerProto c) {
+    return new ContainerPBImpl(c);
+  }
+
+  private ContainerProto convertToProtoFormat(
+      Container c) {
+    return ((ContainerPBImpl)c).getProto();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
index 901051f..b161f5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
@@ -38,6 +38,7 @@ message NodeStatusProto {
   repeated ApplicationIdProto keep_alive_applications = 5;
   optional ResourceUtilizationProto containers_utilization = 6;
   optional ResourceUtilizationProto node_utilization = 7;
+  repeated ContainerProto increased_containers = 8;
 }
 
 message MasterKeyProto {
@@ -60,4 +61,4 @@ message ResourceUtilizationProto {
   optional int32 pmem = 1;
   optional int32 vmem = 2;
   optional float cpu = 3;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index c122b2a..2db8919 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -82,6 +82,7 @@ message NodeHeartbeatResponseProto {
   repeated ContainerIdProto containers_to_be_removed_from_nm = 9;
   repeated SystemCredentialsForAppsProto system_credentials_for_apps = 10;
   optional bool areNodeLabelsAcceptedByRM = 11 [default = false];
+  repeated ContainerProto containers_to_decrease = 12;
 }
 
 message SystemCredentialsForAppsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
index d9eeb9d..c9427dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
@@ -29,6 +29,7 @@ import java.util.HashSet;
 
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -168,6 +169,20 @@ public class TestYarnServerApiClasses {
     assertTrue(copy.getAreNodeLabelsAcceptedByRM());
   }
 
+  @Test
+  public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
+    NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
+    original.addAllContainersToDecrease(
+        Arrays.asList(getDecreasedContainer(1, 2, 2048, 2),
+            getDecreasedContainer(2, 3, 1024, 1)));
+    NodeHeartbeatResponsePBImpl copy =
+        new NodeHeartbeatResponsePBImpl(original.getProto());
+    assertEquals(1, copy.getContainersToDecrease().get(0)
+        .getId().getContainerId());
+    assertEquals(1024, copy.getContainersToDecrease().get(1)
+        .getResource().getMemory());
+  }
+
   /**
    * Test RegisterNodeManagerRequestPBImpl.
    */
@@ -244,6 +259,9 @@ public class TestYarnServerApiClasses {
     original.setNodeHealthStatus(getNodeHealthStatus());
     original.setNodeId(getNodeId());
     original.setResponseId(1);
+    original.setIncreasedContainers(
+        Arrays.asList(getIncreasedContainer(1, 2, 2048, 2),
+            getIncreasedContainer(2, 3, 4096, 3)));
 
     NodeStatusPBImpl copy = new NodeStatusPBImpl(original.getProto());
     assertEquals(3L, copy.getContainersStatuses().get(1).getContainerId()
@@ -252,7 +270,10 @@ public class TestYarnServerApiClasses {
     assertEquals(1000, copy.getNodeHealthStatus().getLastHealthReportTime());
     assertEquals(9090, copy.getNodeId().getPort());
     assertEquals(1, copy.getResponseId());
-
+    assertEquals(1, copy.getIncreasedContainers().get(0)
+        .getId().getContainerId());
+    assertEquals(4096, copy.getIncreasedContainers().get(1)
+        .getResource().getMemory());
   }
 
   @Test
@@ -347,6 +368,22 @@ public class TestYarnServerApiClasses {
     return new ApplicationIdPBImpl(appId.getProto());
   }
 
+  private Container getDecreasedContainer(int containerID,
+      int appAttemptId, int memory, int vCores) {
+    ContainerId containerId = getContainerId(containerID, appAttemptId);
+    Resource capability = Resource.newInstance(memory, vCores);
+    return Container.newInstance(
+        containerId, null, null, capability, null, null);
+  }
+
+  private Container getIncreasedContainer(int containerID,
+      int appAttemptId, int memory, int vCores) {
+    ContainerId containerId = getContainerId(containerID, appAttemptId);
+    Resource capability = Resource.newInstance(memory, vCores);
+    return Container.newInstance(
+        containerId, null, null, capability, null, null);
+  }
+
   private NodeStatus getNodeStatus() {
     NodeStatus status = recordFactory.newRecordInstance(NodeStatus.class);
     status.setContainersStatuses(new ArrayList<ContainerStatus>());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.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/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 52d937b..9c2d1fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -62,6 +62,9 @@ public interface Context {
 
   ConcurrentMap<ContainerId, Container> getContainers();
 
+  ConcurrentMap<ContainerId, org.apache.hadoop.yarn.api.records.Container>
+      getIncreasedContainers();
+
   NMContainerTokenSecretManager getContainerTokenSecretManager();
   
   NMTokenSecretManagerInNM getNMTokenSecretManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 3cf9f1a..184f489 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -439,6 +439,10 @@ public class NodeManager extends CompositeService
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
 
+    protected final ConcurrentMap<ContainerId,
+        org.apache.hadoop.yarn.api.records.Container> increasedContainers =
+            new ConcurrentHashMap<>();
+
     private final NMContainerTokenSecretManager containerTokenSecretManager;
     private final NMTokenSecretManagerInNM nmTokenSecretManager;
     private ContainerManagementProtocol containerManager;
@@ -493,6 +497,12 @@ public class NodeManager extends CompositeService
     }
 
     @Override
+    public ConcurrentMap<ContainerId, org.apache.hadoop.yarn.api.records.Container>
+        getIncreasedContainers() {
+      return this.increasedContainers;
+    }
+
+    @Override
     public NMContainerTokenSecretManager getContainerTokenSecretManager() {
       return this.containerTokenSecretManager;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.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/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 05efc69..4e49040 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -310,18 +310,28 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   @VisibleForTesting
   protected void registerWithRM()
       throws YarnException, IOException {
-    List<NMContainerStatus> containerReports = getNMContainerStatuses();
+    RegisterNodeManagerResponse regNMResponse;
     Set<NodeLabel> nodeLabels = nodeLabelsHandler.getNodeLabelsForRegistration();
-    RegisterNodeManagerRequest request =
-        RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
-            nodeManagerVersionId, containerReports, getRunningApplications(),
-            nodeLabels);
-    if (containerReports != null) {
-      LOG.info("Registering with RM using containers :" + containerReports);
-    }
-    RegisterNodeManagerResponse regNMResponse =
-        resourceTracker.registerNodeManager(request);
-    this.rmIdentifier = regNMResponse.getRMIdentifier();
+ 
+    // Synchronize NM-RM registration with
+    // ContainerManagerImpl#increaseContainersResource and
+    // ContainerManagerImpl#startContainers to avoid race condition
+    // during RM recovery
+    synchronized (this.context) {
+      List<NMContainerStatus> containerReports = getNMContainerStatuses();
+      RegisterNodeManagerRequest request =
+          RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
+              nodeManagerVersionId, containerReports, getRunningApplications(),
+              nodeLabels);
+      if (containerReports != null) {
+        LOG.info("Registering with RM using containers :" + containerReports);
+      }
+      regNMResponse =
+          resourceTracker.registerNodeManager(request);
+      // Make sure rmIdentifier is set before we release the lock
+      this.rmIdentifier = regNMResponse.getRMIdentifier();
+    }
+
     // if the Resource Manager instructs NM to shutdown.
     if (NodeAction.SHUTDOWN.equals(regNMResponse.getNodeAction())) {
       String message =
@@ -418,10 +428,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     List<ContainerStatus> containersStatuses = getContainerStatuses();
     ResourceUtilization containersUtilization = getContainersUtilization();
     ResourceUtilization nodeUtilization = getNodeUtilization();
+    List<org.apache.hadoop.yarn.api.records.Container> increasedContainers
+        = getIncreasedContainers();
     NodeStatus nodeStatus =
         NodeStatus.newInstance(nodeId, responseId, containersStatuses,
           createKeepAliveApplicationList(), nodeHealthStatus,
-          containersUtilization, nodeUtilization);
+          containersUtilization, nodeUtilization, increasedContainers);
 
     return nodeStatus;
   }
@@ -448,6 +460,21 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     return nodeResourceMonitor.getUtilization();
   }
 
+  /* Get the containers whose resource has been increased since last
+   * NM-RM heartbeat.
+   */
+  private List<org.apache.hadoop.yarn.api.records.Container>
+      getIncreasedContainers() {
+    List<org.apache.hadoop.yarn.api.records.Container>
+        increasedContainers = new ArrayList<>(
+            this.context.getIncreasedContainers().values());
+    for (org.apache.hadoop.yarn.api.records.Container
+        container : increasedContainers) {
+      this.context.getIncreasedContainers().remove(container.getId());
+    }
+    return increasedContainers;
+  }
+
   // Iterate through the NMContext and clone and get all the containers'
   // statuses. If it's a completed container, add into the
   // recentlyStoppedContainers collections.
@@ -765,6 +792,14 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               ((NMContext) context)
                 .setSystemCrendentialsForApps(parseCredentials(systemCredentials));
             }
+
+            List<org.apache.hadoop.yarn.api.records.Container>
+                containersToDecrease = response.getContainersToDecrease();
+            if (!containersToDecrease.isEmpty()) {
+              dispatcher.getEventHandler().handle(
+                  new CMgrDecreaseContainersResourceEvent(containersToDecrease)
+              );
+            }
           } catch (ConnectException e) {
             //catch and throw the exception if tried MAX wait time to connect RM
             dispatcher.getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 6fddfa2..76d87b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -751,31 +751,36 @@ public class ContainerManagerImpl extends CompositeService implements
     List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
-    for (StartContainerRequest request : requests.getStartContainerRequests()) {
-      ContainerId containerId = null;
-      try {
-        if (request.getContainerToken() == null ||
-            request.getContainerToken().getIdentifier() == null) {
-          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+    // Synchronize with NodeStatusUpdaterImpl#registerWithRM
+    // to avoid race condition during NM-RM resync (due to RM restart) while a
+    // container is being started, in particular when the container has not yet
+    // been added to the containers map in NMContext.
+    synchronized (this.context) {
+      for (StartContainerRequest request : requests.getStartContainerRequests()) {
+        ContainerId containerId = null;
+        try {
+          if (request.getContainerToken() == null ||
+              request.getContainerToken().getIdentifier() == null) {
+            throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+          }
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(request.getContainerToken());
+          verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
+              containerTokenIdentifier);
+          containerId = containerTokenIdentifier.getContainerID();
+          startContainerInternal(nmTokenIdentifier, containerTokenIdentifier,
+              request);
+          succeededContainers.add(containerId);
+        } catch (YarnException e) {
+          failedContainers.put(containerId, SerializedException.newInstance(e));
+        } catch (InvalidToken ie) {
+          failedContainers.put(containerId, SerializedException.newInstance(ie));
+          throw ie;
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
         }
-        ContainerTokenIdentifier containerTokenIdentifier =
-            BuilderUtils.newContainerTokenIdentifier(request.getContainerToken());
-        verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
-          containerTokenIdentifier);
-        containerId = containerTokenIdentifier.getContainerID();
-        startContainerInternal(nmTokenIdentifier, containerTokenIdentifier,
-          request);
-        succeededContainers.add(containerId);
-      } catch (YarnException e) {
-        failedContainers.put(containerId, SerializedException.newInstance(e));
-      } catch (InvalidToken ie) {
-        failedContainers.put(containerId, SerializedException.newInstance(ie));
-        throw ie;
-      } catch (IOException e) {
-        throw RPCUtil.getRemoteException(e);
       }
     }
-
     return StartContainersResponse.newInstance(getAuxServiceMetaData(),
       succeededContainers, failedContainers);
   }
@@ -957,32 +962,39 @@ public class ContainerManagerImpl extends CompositeService implements
         = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
-    // Process container resource increase requests
-    for (org.apache.hadoop.yarn.api.records.Token token :
-        requests.getContainersToIncrease()) {
-      ContainerId containerId = null;
-      try {
-        if (token.getIdentifier() == null) {
-          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+    // Synchronize with NodeStatusUpdaterImpl#registerWithRM
+    // to avoid race condition during NM-RM resync (due to RM restart) while a
+    // container resource is being increased in NM, in particular when the
+    // increased container has not yet been added to the increasedContainers
+    // map in NMContext.
+    synchronized (this.context) {
+      // Process container resource increase requests
+      for (org.apache.hadoop.yarn.api.records.Token token :
+          requests.getContainersToIncrease()) {
+        ContainerId containerId = null;
+        try {
+          if (token.getIdentifier() == null) {
+            throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+          }
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(token);
+          verifyAndGetContainerTokenIdentifier(token,
+              containerTokenIdentifier);
+          authorizeStartAndResourceIncreaseRequest(
+              nmTokenIdentifier, containerTokenIdentifier, false);
+          containerId = containerTokenIdentifier.getContainerID();
+          // Reuse the startContainer logic to update NMToken,
+          // as container resource increase request will have come with
+          // an updated NMToken.
+          updateNMTokenIdentifier(nmTokenIdentifier);
+          Resource resource = containerTokenIdentifier.getResource();
+          changeContainerResourceInternal(containerId, resource, true);
+          successfullyIncreasedContainers.add(containerId);
+        } catch (YarnException | InvalidToken e) {
+          failedContainers.put(containerId, SerializedException.newInstance(e));
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
         }
-        ContainerTokenIdentifier containerTokenIdentifier =
-            BuilderUtils.newContainerTokenIdentifier(token);
-        verifyAndGetContainerTokenIdentifier(token,
-            containerTokenIdentifier);
-        authorizeStartAndResourceIncreaseRequest(
-            nmTokenIdentifier, containerTokenIdentifier, false);
-        containerId = containerTokenIdentifier.getContainerID();
-        // Reuse the startContainer logic to update NMToken,
-        // as container resource increase request will have come with
-        // an updated NMToken.
-        updateNMTokenIdentifier(nmTokenIdentifier);
-        Resource resource = containerTokenIdentifier.getResource();
-        changeContainerResourceInternal(containerId, resource, true);
-        successfullyIncreasedContainers.add(containerId);
-      } catch (YarnException | InvalidToken e) {
-        failedContainers.put(containerId, SerializedException.newInstance(e));
-      } catch (IOException e) {
-        throw RPCUtil.getRemoteException(e);
       }
     }
     return IncreaseContainersResourceResponse.newInstance(
@@ -1043,6 +1055,16 @@ public class ContainerManagerImpl extends CompositeService implements
           + " is not smaller than the current resource "
           + currentResource.toString());
     }
+    if (increase) {
+      org.apache.hadoop.yarn.api.records.Container increasedContainer =
+          org.apache.hadoop.yarn.api.records.Container.newInstance(
+              containerId, null, null, targetResource, null, null);
+      if (context.getIncreasedContainers().putIfAbsent(containerId,
+          increasedContainer) != null){
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " resource is being increased.");
+      }
+    }
     this.readLock.lock();
     try {
       if (!serviceStopped) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index c22d475..7bde861 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -18,21 +18,35 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -41,8 +55,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
@@ -50,6 +69,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
@@ -57,12 +78,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -87,7 +111,10 @@ public class TestNodeManagerResync {
   private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
   private final NodeManagerEvent resyncEvent =
       new NodeManagerEvent(NodeManagerEventType.RESYNC);
+  private final long DUMMY_RM_IDENTIFIER = 1234;
 
+  protected static Log LOG = LogFactory
+      .getLog(TestNodeManagerResync.class);
 
   @Before
   public void setup() throws UnsupportedFileSystemException {
@@ -209,6 +236,32 @@ public class TestNodeManagerResync {
     nm.stop();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test(timeout=60000)
+  public void testContainerResourceIncreaseIsSynchronizedWithRMResync()
+      throws IOException, InterruptedException, YarnException {
+    NodeManager nm = new TestNodeManager4();
+    YarnConfiguration conf = createNMConfig();
+    conf.setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
+    nm.init(conf);
+    nm.start();
+    // Start a container and make sure it is in RUNNING state
+    ((TestNodeManager4)nm).startContainer();
+    // Simulate a container resource increase in a separate thread
+    ((TestNodeManager4)nm).increaseContainersResource();
+    // Simulate RM restart by sending a RESYNC event
+    LOG.info("Sending out RESYNC event");
+    nm.getNMDispatcher().getEventHandler().handle(
+        new NodeManagerEvent(NodeManagerEventType.RESYNC));
+    try {
+      syncBarrier.await();
+    } catch (BrokenBarrierException e) {
+      e.printStackTrace();
+    }
+    Assert.assertFalse(assertionFailedInThread.get());
+    nm.stop();
+  }
 
   // This is to test when NM gets the resync response from last heart beat, it
   // should be able to send the already-sent-via-last-heart-beat container
@@ -588,6 +641,211 @@ public class TestNodeManagerResync {
       }
     }}
 
+  class TestNodeManager4 extends NodeManager {
+
+    private Thread increaseContainerResourceThread = null;
+
+    @Override
+    protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+        Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+      return new TestNodeStatusUpdaterImpl4(context, dispatcher,
+          healthChecker, metrics);
+    }
+
+    @Override
+    protected ContainerManagerImpl createContainerManager(Context context,
+        ContainerExecutor exec, DeletionService del,
+        NodeStatusUpdater nodeStatusUpdater,
+        ApplicationACLsManager aclsManager,
+        LocalDirsHandlerService dirsHandler) {
+      return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
+          metrics, aclsManager, dirsHandler){
+        @Override
+        public void
+        setBlockNewContainerRequests(boolean blockNewContainerRequests) {
+          // do nothing
+        }
+
+        @Override
+        protected void authorizeGetAndStopContainerRequest(
+            ContainerId containerId, Container container,
+            boolean stopRequest, NMTokenIdentifier identifier)
+            throws YarnException {
+          // do nothing
+        }
+        @Override
+        protected void authorizeUser(UserGroupInformation remoteUgi,
+            NMTokenIdentifier nmTokenIdentifier) {
+          // do nothing
+        }
+        @Override
+        protected void authorizeStartAndResourceIncreaseRequest(
+            NMTokenIdentifier nmTokenIdentifier,
+            ContainerTokenIdentifier containerTokenIdentifier,
+            boolean startRequest) throws YarnException {
+          try {
+            // Sleep 2 seconds to simulate a pro-longed increase action.
+            // If during this time a RESYNC event is sent by RM, the
+            // resync action should block until the increase action is
+            // completed.
+            // See testContainerResourceIncreaseIsSynchronizedWithRMResync()
+            Thread.sleep(2000);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
+        }
+        @Override
+        protected void updateNMTokenIdentifier(
+            NMTokenIdentifier nmTokenIdentifier)
+                throws SecretManager.InvalidToken {
+          // Do nothing
+        }
+        @Override
+        public Map<String, ByteBuffer> getAuxServiceMetaData() {
+          return new HashMap<>();
+        }
+        @Override
+        protected NMTokenIdentifier selectNMTokenIdentifier(
+            UserGroupInformation remoteUgi) {
+          return new NMTokenIdentifier();
+        }
+      };
+    }
+
+    // Start a container in NM
+    public void startContainer()
+        throws IOException, InterruptedException, YarnException {
+      LOG.info("Start a container and wait until it is in RUNNING state");
+      File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+      PrintWriter fileWriter = new PrintWriter(scriptFile);
+      if (Shell.WINDOWS) {
+        fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+      } else {
+        fileWriter.write("\numask 0");
+        fileWriter.write("\nexec sleep 100");
+      }
+      fileWriter.close();
+      ContainerLaunchContext containerLaunchContext =
+          recordFactory.newRecordInstance(ContainerLaunchContext.class);
+      URL resource_alpha =
+          ConverterUtils.getYarnUrlFromPath(localFS
+              .makeQualified(new Path(scriptFile.getAbsolutePath())));
+      LocalResource rsrc_alpha =
+          recordFactory.newRecordInstance(LocalResource.class);
+      rsrc_alpha.setResource(resource_alpha);
+      rsrc_alpha.setSize(-1);
+      rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+      rsrc_alpha.setType(LocalResourceType.FILE);
+      rsrc_alpha.setTimestamp(scriptFile.lastModified());
+      String destinationFile = "dest_file";
+      Map<String, LocalResource> localResources =
+          new HashMap<String, LocalResource>();
+      localResources.put(destinationFile, rsrc_alpha);
+      containerLaunchContext.setLocalResources(localResources);
+      List<String> commands =
+          Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+      containerLaunchContext.setCommands(commands);
+      Resource resource = Resource.newInstance(1024, 1);
+      StartContainerRequest scRequest =
+          StartContainerRequest.newInstance(
+              containerLaunchContext,
+              getContainerToken(resource));
+      List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+      list.add(scRequest);
+      StartContainersRequest allRequests =
+          StartContainersRequest.newInstance(list);
+      getContainerManager().startContainers(allRequests);
+      // Make sure the container reaches RUNNING state
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      BaseContainerManagerTest.waitForNMContainerState(
+          getContainerManager(), cId,
+          org.apache.hadoop.yarn.server.nodemanager.
+              containermanager.container.ContainerState.RUNNING);
+    }
+
+    // Increase container resource in a thread
+    public void increaseContainersResource()
+        throws InterruptedException {
+      LOG.info("Increase a container resource in a separate thread");
+      increaseContainerResourceThread = new IncreaseContainersResourceThread();
+      increaseContainerResourceThread.start();
+    }
+
+    class TestNodeStatusUpdaterImpl4 extends MockNodeStatusUpdater {
+
+      public TestNodeStatusUpdaterImpl4(Context context, Dispatcher dispatcher,
+          NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+        super(context, dispatcher, healthChecker, metrics);
+      }
+
+      @Override
+      protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
+        try {
+          try {
+            // Check status before registerWithRM
+            List<ContainerId> containerIds = new ArrayList<>();
+            ContainerId cId = TestContainerManager.createContainerId(0);
+            containerIds.add(cId);
+            GetContainerStatusesRequest gcsRequest =
+                GetContainerStatusesRequest.newInstance(containerIds);
+            ContainerStatus containerStatus = getContainerManager()
+                .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+            assertEquals(Resource.newInstance(1024, 1),
+                containerStatus.getCapability());
+            // Call the actual rebootNodeStatusUpdaterAndRegisterWithRM().
+            // This function should be synchronized with
+            // increaseContainersResource().
+            super.rebootNodeStatusUpdaterAndRegisterWithRM();
+            // Check status after registerWithRM
+            containerStatus = getContainerManager()
+                .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+            assertEquals(Resource.newInstance(4096, 2),
+                containerStatus.getCapability());
+          } catch (AssertionError ae) {
+            ae.printStackTrace();
+            assertionFailedInThread.set(true);
+          }   finally {
+            syncBarrier.await();
+          }
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    class IncreaseContainersResourceThread extends Thread {
+      @Override
+      public void run() {
+        // Construct container resource increase request
+        List<Token> increaseTokens = new ArrayList<Token>();
+        // Add increase request.
+        Resource targetResource = Resource.newInstance(4096, 2);
+        try {
+          increaseTokens.add(getContainerToken(targetResource));
+          IncreaseContainersResourceRequest increaseRequest =
+              IncreaseContainersResourceRequest.newInstance(increaseTokens);
+          IncreaseContainersResourceResponse increaseResponse =
+              getContainerManager()
+                  .increaseContainersResource(increaseRequest);
+          Assert.assertEquals(
+              1, increaseResponse.getSuccessfullyIncreasedContainers()
+                  .size());
+          Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    private Token getContainerToken(Resource resource) throws IOException {
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      return TestContainerManager.createContainerToken(
+          cId, DUMMY_RM_IDENTIFIER,
+          getNMContext().getNodeId(), user, resource,
+          getNMContext().getContainerTokenSecretManager(), null);
+    }
+  }
+
   public static NMContainerStatus createNMContainerStatus(int id,
       ContainerState containerState) {
     ApplicationId applicationId = ApplicationId.newInstance(0, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf3d5e54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 2ea9146..3fb4112 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -108,7 +108,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     super.setup();
   }
 
-  private ContainerId createContainerId(int id) {
+  public static ContainerId createContainerId(int id) {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);


[12/50] [abbrv] hadoop git commit: YARN-4082. Container shouldn't be killed when node's label updated. Contributed by Wangda Tan.

Posted by wa...@apache.org.
YARN-4082. Container shouldn't be killed when node's label updated. Contributed by Wangda Tan.


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

Branch: refs/heads/YARN-1197
Commit: bf669b6d9f8ba165e30b8823218d625a49958925
Parents: f4d96be
Author: Varun Vasudev <vv...@apache.org>
Authored: Tue Sep 1 14:19:11 2015 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Tue Sep 1 14:19:11 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/capacity/AbstractCSQueue.java     |  27 ++
 .../scheduler/capacity/CSQueue.java             |  26 ++
 .../scheduler/capacity/CapacityScheduler.java   |  40 +--
 .../scheduler/capacity/LeafQueue.java           |  16 ++
 .../scheduler/common/fica/FiCaSchedulerApp.java |   9 +
 .../TestCapacitySchedulerNodeLabelUpdate.java   | 249 ++++++++++++++++---
 7 files changed, 314 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 80cf793..999654d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -804,6 +804,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3896. RMNode transitioned from RUNNING to REBOOTED because its response id 
     has not been reset synchronously. (Jun Gong via rohithsharmaks)
 
+    YARN-4082. Container shouldn't be killed when node's label updated.
+    (Wangda Tan via vvasudev)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index 792c25c..0ae4d1a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -543,6 +544,32 @@ public abstract class AbstractCSQueue implements CSQueue {
     }
   }
   
+  @Override
+  public void incUsedResource(String nodeLabel, Resource resourceToInc,
+      SchedulerApplicationAttempt application) {
+    if (nodeLabel == null) {
+      nodeLabel = RMNodeLabelsManager.NO_LABEL;
+    }
+    // ResourceUsage has its own lock, no addition lock needs here.
+    queueUsage.incUsed(nodeLabel, resourceToInc);
+    if (null != parent) {
+      parent.incUsedResource(nodeLabel, resourceToInc, null);
+    }
+  }
+
+  @Override
+  public void decUsedResource(String nodeLabel, Resource resourceToDec,
+      SchedulerApplicationAttempt application) {
+    if (nodeLabel == null) {
+      nodeLabel = RMNodeLabelsManager.NO_LABEL;
+    }
+    // ResourceUsage has its own lock, no addition lock needs here.
+    queueUsage.decUsed(nodeLabel, resourceToDec);
+    if (null != parent) {
+      parent.decUsedResource(nodeLabel, resourceToDec, null);
+    }
+  }
+
   /**
    * Return if the queue has pending resource on given nodePartition and
    * schedulingMode. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
index b06a646..9855dd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 
@@ -287,4 +288,29 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
    * @return resourceUsage
    */
   public ResourceUsage getQueueResourceUsage();
+
+  /**
+   * When partition of node updated, we will update queue's resource usage if it
+   * has container(s) running on that.
+   */
+  public void incUsedResource(String nodePartition, Resource resourceToInc,
+      SchedulerApplicationAttempt application);
+
+  /**
+   * When partition of node updated, we will update queue's resource usage if it
+   * has container(s) running on that.
+   */
+  public void decUsedResource(String nodePartition, Resource resourceToDec,
+      SchedulerApplicationAttempt application);
+
+  /**
+   * When an outstanding resource is fulfilled or canceled, calling this will
+   * decrease pending resource in a queue.
+   *
+   * @param nodeLabel
+   *          asked by application
+   * @param resourceToDec
+   *          new resource asked
+   */
+  public void decPendingResource(String nodeLabel, Resource resourceToDec);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index cff1fe5..b5ccbd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -1040,12 +1040,6 @@ public class CapacityScheduler extends
   
   /**
    * Process node labels update on a node.
-   * 
-   * TODO: Currently capacity scheduler will kill containers on a node when
-   * labels on the node changed. It is a simply solution to ensure guaranteed
-   * capacity on labels of queues. When YARN-2498 completed, we can let
-   * preemption policy to decide if such containers need to be killed or just
-   * keep them running.
    */
   private synchronized void updateLabelsOnNode(NodeId nodeId,
       Set<String> newLabels) {
@@ -1060,17 +1054,31 @@ public class CapacityScheduler extends
       return;
     }
     
-    // Kill running containers since label is changed
+    // Get new partition, we have only one partition per node
+    String newPartition;
+    if (newLabels.isEmpty()) {
+      newPartition = RMNodeLabelsManager.NO_LABEL;
+    } else {
+      newPartition = newLabels.iterator().next();
+    }
+
+    // old partition as well
+    String oldPartition = node.getPartition();
+
+    // Update resources of these containers
     for (RMContainer rmContainer : node.getRunningContainers()) {
-      ContainerId containerId = rmContainer.getContainerId();
-      completedContainer(rmContainer, 
-          ContainerStatus.newInstance(containerId,
-              ContainerState.COMPLETE, 
-              String.format(
-                  "Container=%s killed since labels on the node=%s changed",
-                  containerId.toString(), nodeId.toString()),
-              ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
-          RMContainerEventType.KILL);
+      FiCaSchedulerApp application =
+          getApplicationAttempt(rmContainer.getApplicationAttemptId());
+      if (null != application) {
+        application.nodePartitionUpdated(rmContainer, oldPartition,
+            newPartition);
+      } else {
+        LOG.warn("There's something wrong, some RMContainers running on"
+            + " a node, but we cannot find SchedulerApplicationAttempt for it. Node="
+            + node.getNodeID() + " applicationAttemptId="
+            + rmContainer.getApplicationAttemptId());
+        continue;
+      }
     }
     
     // Unreserve container on this node

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index ff1baff..658eae1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -1262,6 +1262,22 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
+  @Override
+  public void incUsedResource(String nodeLabel, Resource resourceToInc,
+      SchedulerApplicationAttempt application) {
+    getUser(application.getUser()).getResourceUsage().incUsed(nodeLabel,
+        resourceToInc);
+    super.incUsedResource(nodeLabel, resourceToInc, application);
+  }
+
+  @Override
+  public void decUsedResource(String nodeLabel, Resource resourceToDec,
+      SchedulerApplicationAttempt application) {
+    getUser(application.getUser()).getResourceUsage().decUsed(nodeLabel,
+        resourceToDec);
+    super.decUsedResource(nodeLabel, resourceToDec, application);
+  }
+
   @VisibleForTesting
   public static class User {
     ResourceUsage userResourceUsage = new ResourceUsage();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 74d77f5..300cba9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -443,4 +443,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           schedulingMode, currentResourceLimits, reservedContainer);
     }
   }
+
+  public void nodePartitionUpdated(RMContainer rmContainer, String oldPartition,
+      String newPartition) {
+    Resource containerResource = rmContainer.getAllocatedResource();
+    this.attemptResourceUsage.decUsed(oldPartition, containerResource);
+    this.attemptResourceUsage.incUsed(newPartition, containerResource);
+    getCSLeafQueue().decUsedResource(oldPartition, containerResource, this);
+    getCSLeafQueue().incUsedResource(newPartition, containerResource, this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bf669b6d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
index 0a701d8..94af4e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerNodeLabelUpdate.java
@@ -19,22 +19,29 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
 import org.junit.Assert;
 import org.junit.Before;
@@ -97,8 +104,18 @@ public class TestCapacitySchedulerNodeLabelUpdate {
         .getMemory());
   }
 
+  private void checkUserUsedResource(MockRM rm, String queueName,
+      String userName, String partition, int memory) {
+    CapacityScheduler scheduler = (CapacityScheduler) rm.getResourceScheduler();
+    LeafQueue queue = (LeafQueue) scheduler.getQueue(queueName);
+    LeafQueue.User user = queue.getUser(userName);
+    Assert.assertEquals(memory,
+        user.getResourceUsage().getUsed(partition).getMemory());
+  }
+
   @Test(timeout = 60000)
-  public void testResourceUsage() throws Exception {
+  public void testRequestContainerAfterNodePartitionUpdated()
+      throws Exception {
     // set node -> label
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y",
         "z"));
@@ -160,7 +177,8 @@ public class TestCapacitySchedulerNodeLabelUpdate {
   }
 
   @Test (timeout = 60000)
-  public void testNodeUpdate() throws Exception {
+  public void testResourceUsageWhenNodeUpdatesPartition()
+      throws Exception {
     // set node -> label
     mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z"));
     
@@ -183,8 +201,9 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     MockNM nm1 = rm.registerNode("h1:1234", 8000);
     MockNM nm2 = rm.registerNode("h2:1234", 8000);
     MockNM nm3 = rm.registerNode("h3:1234", 8000);
-    
-    ContainerId containerId;
+
+    ContainerId containerId1;
+    ContainerId containerId2;
 
     // launch an app to queue a1 (label = x), and check all container will
     // be allocated in h1
@@ -193,9 +212,9 @@ public class TestCapacitySchedulerNodeLabelUpdate {
 
     // request a container.
     am1.allocate("*", GB, 1, new ArrayList<ContainerId>(), "x");
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
-    Assert.assertTrue(rm.waitForState(nm1, containerId,
+    containerId1 = ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+    containerId2 = ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm.waitForState(nm1, containerId2,
         RMContainerState.ALLOCATED, 10 * 1000));
     
     // check used resource:
@@ -203,55 +222,205 @@ public class TestCapacitySchedulerNodeLabelUpdate {
     checkUsedResource(rm, "a", 1024, "x");
     checkUsedResource(rm, "a", 1024);
     
-    // change h1's label to z, container should be killed
-    mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
-        toSet("z")));
-    Assert.assertTrue(rm.waitForState(nm1, containerId,
-        RMContainerState.KILLED, 10 * 1000));
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    FiCaSchedulerApp app = cs.getApplicationAttempt(am1.getApplicationAttemptId());
     
-    // check used resource:
-    // queue-a used x=0G, ""=1G ("" not changed)
+    // change h1's label to z
+    cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(),
+        toSet("z"))));
     checkUsedResource(rm, "a", 0, "x");
+    checkUsedResource(rm, "a", 1024, "z");
     checkUsedResource(rm, "a", 1024);
+    checkUsedResource(rm, "root", 0, "x");
+    checkUsedResource(rm, "root", 1024, "z");
+    checkUsedResource(rm, "root", 1024);
+    checkUserUsedResource(rm, "a", "user", "x", 0);
+    checkUserUsedResource(rm, "a", "user", "z", 1024);
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("x").getMemory());
+    Assert.assertEquals(1024,
+        app.getAppAttemptResourceUsage().getUsed("z").getMemory());
     
-    // request a container with label = y
-    am1.allocate("*", GB, 1, new ArrayList<ContainerId>(), "y");
-    containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
-    Assert.assertTrue(rm.waitForState(nm2, containerId,
-        RMContainerState.ALLOCATED, 10 * 1000));
-    
-    // check used resource:
-    // queue-a used y=1G, ""=1G
+    // change h1's label to y
+    cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(),
+        toSet("y"))));
+    checkUsedResource(rm, "a", 0, "x");
     checkUsedResource(rm, "a", 1024, "y");
+    checkUsedResource(rm, "a", 0, "z");
     checkUsedResource(rm, "a", 1024);
+    checkUsedResource(rm, "root", 0, "x");
+    checkUsedResource(rm, "root", 1024, "y");
+    checkUsedResource(rm, "root", 0, "z");
+    checkUsedResource(rm, "root", 1024);
+    checkUserUsedResource(rm, "a", "user", "x", 0);
+    checkUserUsedResource(rm, "a", "user", "y", 1024);
+    checkUserUsedResource(rm, "a", "user", "z", 0);
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("x").getMemory());
+    Assert.assertEquals(1024,
+        app.getAppAttemptResourceUsage().getUsed("y").getMemory());
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("z").getMemory());
     
-    // change h2's label to no label, container should be killed
-    mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h2", 0),
-        CommonNodeLabelsManager.EMPTY_STRING_SET));
-    Assert.assertTrue(rm.waitForState(nm1, containerId,
-        RMContainerState.KILLED, 10 * 1000));
+    // change h1's label to no label
+    Set<String> emptyLabels = new HashSet<>();
+    Map<NodeId,Set<String>> map = ImmutableMap.of(nm1.getNodeId(),
+        emptyLabels);
+    cs.handle(new NodeLabelsUpdateSchedulerEvent(map));
+    checkUsedResource(rm, "a", 0, "x");
+    checkUsedResource(rm, "a", 0, "y");
+    checkUsedResource(rm, "a", 0, "z");
+    checkUsedResource(rm, "a", 2048);
+    checkUsedResource(rm, "root", 0, "x");
+    checkUsedResource(rm, "root", 0, "y");
+    checkUsedResource(rm, "root", 0, "z");
+    checkUsedResource(rm, "root", 2048);
+    checkUserUsedResource(rm, "a", "user", "x", 0);
+    checkUserUsedResource(rm, "a", "user", "y", 0);
+    checkUserUsedResource(rm, "a", "user", "z", 0);
+    checkUserUsedResource(rm, "a", "user", "", 2048);
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("x").getMemory());
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("y").getMemory());
+    Assert.assertEquals(0,
+        app.getAppAttemptResourceUsage().getUsed("z").getMemory());
+    Assert.assertEquals(2048,
+        app.getAppAttemptResourceUsage().getUsed("").getMemory());
+
+    // Finish the two containers, we should see used resource becomes 0
+    cs.completedContainer(cs.getRMContainer(containerId2),
+        ContainerStatus.newInstance(containerId2, ContainerState.COMPLETE, "",
+            ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+        RMContainerEventType.KILL);
+    cs.completedContainer(cs.getRMContainer(containerId1),
+        ContainerStatus.newInstance(containerId1, ContainerState.COMPLETE, "",
+            ContainerExitStatus.KILLED_BY_RESOURCEMANAGER),
+        RMContainerEventType.KILL);
     
-    // check used resource:
-    // queue-a used x=0G, y=0G, ""=1G ("" not changed)
     checkUsedResource(rm, "a", 0, "x");
     checkUsedResource(rm, "a", 0, "y");
-    checkUsedResource(rm, "a", 1024);
+    checkUsedResource(rm, "a", 0, "z");
+    checkUsedResource(rm, "a", 0);
+    checkUsedResource(rm, "root", 0, "x");
+    checkUsedResource(rm, "root", 0, "y");
+    checkUsedResource(rm, "root", 0, "z");
+    checkUsedResource(rm, "root", 0);
+    checkUserUsedResource(rm, "a", "user", "x", 0);
+    checkUserUsedResource(rm, "a", "user", "y", 0);
+    checkUserUsedResource(rm, "a", "user", "z", 0);
+    checkUserUsedResource(rm, "a", "user", "", 0);
+
+    rm.close();
+  }
+
+
+  @Test (timeout = 60000)
+  public void testComplexResourceUsageWhenNodeUpdatesPartition()
+      throws Exception {
+    /*
+     * This test is similar to testResourceUsageWhenNodeUpdatesPartition, this
+     * will include multiple applications, multiple users and multiple
+     * containers running on a single node, size of each container is 1G
+     *
+     * Node 1
+     * ------
+     * App1-container3
+     * App2-container2
+     * App2-Container3
+     *
+     * Node 2
+     * ------
+     * App2-container1
+     * App1-container1
+     * App1-container2
+     */
+    // set node -> label
+    mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y", "z"));
     
+    // set mapping:
+    // h1 -> x
+    // h2 -> y
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+
+    // inject node label manager
+    MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) {
+      @Override
+      public RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+
+    rm.getRMContext().setNodeLabelManager(mgr);
+    rm.start();
+    MockNM nm1 = rm.registerNode("h1:1234", 80000);
+    MockNM nm2 = rm.registerNode("h2:1234", 80000);
+
+    // app1
+    RMApp app1 = rm.submitApp(GB, "app", "u1", null, "a");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+
+    // c2 on n1, c3 on n2
+    am1.allocate("*", GB, 1, new ArrayList<ContainerId>(), "x");
+    ContainerId containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    am1.allocate("*", GB, 1, new ArrayList<ContainerId>());
     containerId =
-        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
+    Assert.assertTrue(rm.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
     
-    // change h3's label to z, AM container should be killed
-    mgr.replaceLabelsOnNode(ImmutableMap.of(NodeId.newInstance("h3", 0),
-        toSet("z")));
+    // app2
+    RMApp app2 = rm.submitApp(GB, "app", "u2", null, "a");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+
+    // c2/c3 on n1
+    am2.allocate("*", GB, 2, new ArrayList<ContainerId>(), "x");
+    containerId =
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 3);
     Assert.assertTrue(rm.waitForState(nm1, containerId,
-        RMContainerState.KILLED, 10 * 1000));
+        RMContainerState.ALLOCATED, 10 * 1000));
     
     // check used resource:
-    // queue-a used x=0G, y=0G, ""=1G ("" not changed)
+    // queue-a used x=1G, ""=1G
+    checkUsedResource(rm, "a", 3 * GB, "x");
+    checkUsedResource(rm, "a", 3 * GB);
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    FiCaSchedulerApp application1 =
+        cs.getApplicationAttempt(am1.getApplicationAttemptId());
+    FiCaSchedulerApp application2 =
+        cs.getApplicationAttempt(am2.getApplicationAttemptId());
+
+    // change h1's label to z
+    cs.handle(new NodeLabelsUpdateSchedulerEvent(ImmutableMap.of(nm1.getNodeId(),
+        toSet("z"))));
     checkUsedResource(rm, "a", 0, "x");
-    checkUsedResource(rm, "a", 0, "y");
-    checkUsedResource(rm, "a", 0);
+    checkUsedResource(rm, "a", 3 * GB, "z");
+    checkUsedResource(rm, "a", 3 * GB);
+    checkUsedResource(rm, "root", 0, "x");
+    checkUsedResource(rm, "root", 3 * GB, "z");
+    checkUsedResource(rm, "root", 3 * GB);
+    checkUserUsedResource(rm, "a", "u1", "x", 0 * GB);
+    checkUserUsedResource(rm, "a", "u1", "z", 1 * GB);
+    checkUserUsedResource(rm, "a", "u1", "", 2 * GB);
+    checkUserUsedResource(rm, "a", "u2", "x", 0 * GB);
+    checkUserUsedResource(rm, "a", "u2", "z", 2 * GB);
+    checkUserUsedResource(rm, "a", "u2", "", 1 * GB);
+    Assert.assertEquals(0,
+        application1.getAppAttemptResourceUsage().getUsed("x").getMemory());
+    Assert.assertEquals(1 * GB,
+        application1.getAppAttemptResourceUsage().getUsed("z").getMemory());
+    Assert.assertEquals(2 * GB,
+        application1.getAppAttemptResourceUsage().getUsed("").getMemory());
+    Assert.assertEquals(0,
+        application2.getAppAttemptResourceUsage().getUsed("x").getMemory());
+    Assert.assertEquals(2 * GB,
+        application2.getAppAttemptResourceUsage().getUsed("z").getMemory());
+    Assert.assertEquals(1 * GB,
+        application2.getAppAttemptResourceUsage().getUsed("").getMemory());
 
     rm.close();
   }


[38/50] [abbrv] hadoop git commit: HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: ed78b14ebc9a21bb57ccd088e8b49bfa457a396f
Parents: c2d2c18
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Sep 3 15:32:53 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Sep 3 15:32:53 2015 -0700

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |   1 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  64 +++
 .../hdfs/client/HdfsClientConfigKeys.java       |  14 +
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   | 133 +++++
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   | 142 +++++
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java | 136 +++++
 .../java/org/apache/hadoop/hdfs/net/Peer.java   |   8 +-
 .../protocol/datatransfer/IOStreamPair.java     |  37 ++
 .../datatransfer/TrustedChannelResolver.java    |  81 +++
 .../sasl/DataEncryptionKeyFactory.java          |  38 ++
 .../datatransfer/sasl/DataTransferSaslUtil.java | 519 +++++++++++++++++++
 .../sasl/SaslDataTransferClient.java            | 498 ++++++++++++++++++
 .../datatransfer/sasl/SaslParticipant.java      | 210 ++++++++
 .../SaslResponseWithNegotiatedCipherOption.java |  33 ++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 102 ++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   3 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  28 +-
 .../apache/hadoop/hdfs/net/BasicInetPeer.java   | 133 -----
 .../apache/hadoop/hdfs/net/EncryptedPeer.java   | 142 -----
 .../org/apache/hadoop/hdfs/net/NioInetPeer.java | 136 -----
 .../apache/hadoop/hdfs/net/TcpPeerServer.java   |  65 +--
 .../protocol/datatransfer/IOStreamPair.java     |  37 --
 .../datatransfer/TrustedChannelResolver.java    |  81 ---
 .../sasl/DataEncryptionKeyFactory.java          |  38 --
 .../datatransfer/sasl/DataTransferSaslUtil.java | 519 -------------------
 .../sasl/SaslDataTransferClient.java            | 498 ------------------
 .../sasl/SaslDataTransferServer.java            |   2 +-
 .../datatransfer/sasl/SaslParticipant.java      | 210 --------
 .../SaslResponseWithNegotiatedCipherOption.java |  33 --
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 111 +---
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   8 +-
 .../hdfs/server/namenode/FSDirectory.java       |   3 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../apache/hadoop/hdfs/BlockReaderTestUtil.java |   3 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |   2 +-
 .../hadoop/hdfs/TestEncryptedTransfer.java      |   4 +-
 .../sasl/SaslDataTransferTestCase.java          |   2 +-
 .../datatransfer/sasl/TestSaslDataTransfer.java |   2 +-
 .../hdfs/qjournal/TestSecureNNWithQJM.java      |   2 +-
 .../blockmanagement/TestBlockTokenWithDFS.java  |   4 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   4 +-
 42 files changed, 2064 insertions(+), 2029 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 036ac09..515da24 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -14,6 +14,7 @@
       <Class name="org.apache.hadoop.hdfs.protocol.SnapshotDiffReport$DiffReportEntry"/>
       <Class name="org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus"/>
       <Class name="org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslResponseWithNegotiatedCipherOption"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index a89f556..b032250 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -27,16 +27,24 @@ import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.net.BasicInetPeer;
+import org.apache.hadoop.hdfs.net.NioInetPeer;
+import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,8 +54,10 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.Socket;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.nio.channels.SocketChannel;
 import java.text.SimpleDateFormat;
 import java.util.Collection;
 import java.util.Collections;
@@ -523,4 +533,58 @@ public class DFSUtilClient {
     }
     return keyProvider;
   }
+
+  public static Peer peerFromSocket(Socket socket)
+      throws IOException {
+    Peer peer = null;
+    boolean success = false;
+    try {
+      // TCP_NODELAY is crucial here because of bad interactions between
+      // Nagle's Algorithm and Delayed ACKs. With connection keepalive
+      // between the client and DN, the conversation looks like:
+      //   1. Client -> DN: Read block X
+      //   2. DN -> Client: data for block X
+      //   3. Client -> DN: Status OK (successful read)
+      //   4. Client -> DN: Read block Y
+      // The fact that step #3 and #4 are both in the client->DN direction
+      // triggers Nagling. If the DN is using delayed ACKs, this results
+      // in a delay of 40ms or more.
+      //
+      // TCP_NODELAY disables nagling and thus avoids this performance
+      // disaster.
+      socket.setTcpNoDelay(true);
+      SocketChannel channel = socket.getChannel();
+      if (channel == null) {
+        peer = new BasicInetPeer(socket);
+      } else {
+        peer = new NioInetPeer(socket);
+      }
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        if (peer != null) peer.close();
+        socket.close();
+      }
+    }
+  }
+
+  public static Peer peerFromSocketAndKey(
+        SaslDataTransferClient saslClient, Socket s,
+        DataEncryptionKeyFactory keyFactory,
+        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
+        throws IOException {
+    Peer peer = null;
+    boolean success = false;
+    try {
+      peer = peerFromSocket(s);
+      peer = saslClient.peerSend(peer, keyFactory, blockToken, datanodeId);
+      success = true;
+      return peer;
+    } finally {
+      if (!success) {
+        IOUtilsClient.cleanup(null, peer);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 96bc8d3..e417fbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -126,6 +126,20 @@ public interface HdfsClientConfigKeys {
   long    DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
   String  DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
 
+  String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY =
+      "dfs.encrypt.data.transfer.cipher.suites";
+
+  String DFS_DATA_TRANSFER_PROTECTION_KEY = "dfs.data.transfer.protection";
+  String DFS_DATA_TRANSFER_PROTECTION_DEFAULT = "";
+  String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY =
+      "dfs.data.transfer.saslproperties.resolver.class";
+
+  String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY =
+      "dfs.encrypt.data.transfer.cipher.key.bitlength";
+  int    DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
+
+  String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
+
   String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
       PREFIX + "replica.accessor.builder.classes";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
new file mode 100644
index 0000000..212dbef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
@@ -0,0 +1,133 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.net.unix.DomainSocket;
+
+/**
+ * Represents a peer that we communicate with by using a basic Socket
+ * that has no associated Channel.
+ *
+ */
+public class BasicInetPeer implements Peer {
+  private final Socket socket;
+  private final OutputStream out;
+  private final InputStream in;
+  private final boolean isLocal;
+
+  public BasicInetPeer(Socket socket) throws IOException {
+    this.socket = socket;
+    this.out = socket.getOutputStream();
+    this.in = socket.getInputStream();
+    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    /*
+     * This Socket has no channel, so there's nothing to return here.
+     */
+    return null;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    socket.setSoTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return socket.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return socket.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) {
+   /* 
+    * We can't implement write timeouts. :(
+    * 
+    * Java provides no facility to set a blocking write timeout on a Socket.
+    * You can simulate a blocking write with a timeout by using
+    * non-blocking I/O.  However, we can't use nio here, because this Socket
+    * doesn't have an associated Channel.
+    * 
+    * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
+    * more details.
+    */
+  }
+
+  @Override
+  public boolean isClosed() {
+    return socket.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    socket.close();
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return socket.getRemoteSocketAddress().toString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return socket.getLocalSocketAddress().toString();
+  }
+  
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+
+  @Override
+  public String toString() {
+    return "BasicInetPeer(" + socket.toString() + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return null;
+  }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
new file mode 100644
index 0000000..da660c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
@@ -0,0 +1,142 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.net.unix.DomainSocket;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.channels.ReadableByteChannel;
+
+/**
+ * Represents a peer that we communicate with by using an encrypted
+ * communications medium.
+ */
+@InterfaceAudience.Private
+public class EncryptedPeer implements Peer {
+  private final Peer enclosedPeer;
+
+  /**
+   * An encrypted InputStream.
+   */
+  private final InputStream in;
+  
+  /**
+   * An encrypted OutputStream.
+   */
+  private final OutputStream out;
+  
+  /**
+   * An encrypted ReadableByteChannel.
+   */
+  private final ReadableByteChannel channel;
+
+  public EncryptedPeer(Peer enclosedPeer, IOStreamPair ios) {
+    this.enclosedPeer = enclosedPeer;
+    this.in = ios.in;
+    this.out = ios.out;
+    this.channel = ios.in instanceof ReadableByteChannel ? 
+        (ReadableByteChannel)ios.in : null;
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    return channel;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    enclosedPeer.setReadTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return enclosedPeer.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return enclosedPeer.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) throws IOException {
+    enclosedPeer.setWriteTimeout(timeoutMs);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return enclosedPeer.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      in.close();
+    } finally {
+      try {
+        out.close();
+      } finally {
+        enclosedPeer.close();
+      }
+    }
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return enclosedPeer.getRemoteAddressString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return enclosedPeer.getLocalAddressString();
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return enclosedPeer.isLocal();
+  }
+
+  @Override
+  public String toString() {
+    return "EncryptedPeer(" + enclosedPeer + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return enclosedPeer.getDomainSocket();
+  }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
new file mode 100644
index 0000000..a12a69b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
@@ -0,0 +1,136 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.Socket;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.net.SocketInputStream;
+import org.apache.hadoop.net.SocketOutputStream;
+import org.apache.hadoop.net.unix.DomainSocket;
+
+/**
+ * Represents a peer that we communicate with by using non-blocking I/O 
+ * on a Socket.
+ */
+public class NioInetPeer implements Peer {
+  private final Socket socket;
+
+  /**
+   * An InputStream which simulates blocking I/O with timeouts using NIO.
+   */
+  private final SocketInputStream in;
+  
+  /**
+   * An OutputStream which simulates blocking I/O with timeouts using NIO.
+   */
+  private final SocketOutputStream out;
+
+  private final boolean isLocal;
+
+  public NioInetPeer(Socket socket) throws IOException {
+    this.socket = socket;
+    this.in = new SocketInputStream(socket.getChannel(), 0);
+    this.out = new SocketOutputStream(socket.getChannel(), 0);
+    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
+  }
+
+  @Override
+  public ReadableByteChannel getInputStreamChannel() {
+    return in;
+  }
+
+  @Override
+  public void setReadTimeout(int timeoutMs) throws IOException {
+    in.setTimeout(timeoutMs);
+  }
+
+  @Override
+  public int getReceiveBufferSize() throws IOException {
+    return socket.getReceiveBufferSize();
+  }
+
+  @Override
+  public boolean getTcpNoDelay() throws IOException {
+    return socket.getTcpNoDelay();
+  }
+
+  @Override
+  public void setWriteTimeout(int timeoutMs) throws IOException {
+    out.setTimeout(timeoutMs);
+  }
+
+  @Override
+  public boolean isClosed() {
+    return socket.isClosed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    // We always close the outermost streams-- in this case, 'in' and 'out'
+    // Closing either one of these will also close the Socket.
+    try {
+      in.close();
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public String getRemoteAddressString() {
+    return socket.getRemoteSocketAddress().toString();
+  }
+
+  @Override
+  public String getLocalAddressString() {
+    return socket.getLocalSocketAddress().toString();
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return in;
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return out;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+
+  @Override
+  public String toString() {
+    return "NioInetPeer(" + socket.toString() + ")";
+  }
+
+  @Override
+  public DomainSocket getDomainSocket() {
+    return null;
+  }
+
+  @Override
+  public boolean hasSecureChannel() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
index 42cf287..3c38d5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
@@ -57,8 +57,8 @@ public interface Peer extends Closeable {
    * Set the write timeout on this peer.
    *
    * Note: this is not honored for BasicInetPeer.
-   * See {@link BasicSocketPeer#setWriteTimeout} for details.
-   * 
+   * See {@link BasicInetPeer#setWriteTimeout} for details.
+   *
    * @param timeoutMs       The timeout in milliseconds.
    */
   public void setWriteTimeout(int timeoutMs) throws IOException;
@@ -76,13 +76,13 @@ public interface Peer extends Closeable {
   public void close() throws IOException;
 
   /**
-   * @return               A string representing the remote end of our 
+   * @return               A string representing the remote end of our
    *                       connection to the peer.
    */
   public String getRemoteAddressString();
 
   /**
-   * @return               A string representing the local end of our 
+   * @return               A string representing the local end of our
    *                       connection to the peer.
    */
   public String getLocalAddressString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
new file mode 100644
index 0000000..23407f8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.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.
+ */
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A little struct class to wrap an InputStream and an OutputStream.
+ */
+@InterfaceAudience.Private
+public class IOStreamPair {
+  public final InputStream in;
+  public final OutputStream out;
+  
+  public IOStreamPair(InputStream in, OutputStream out) {
+    this.in = in;
+    this.out = out;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
new file mode 100644
index 0000000..3846f4a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
@@ -0,0 +1,81 @@
+/**
+ * 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.protocol.datatransfer;
+
+import java.net.InetAddress;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Class used to indicate whether a channel is trusted or not.
+ * The default implementation is to return false indicating that
+ * the channel is not trusted.
+ * This class can be overridden to provide custom logic to determine
+ * whether a channel is trusted or not. 
+ * The custom class can be specified via configuration.
+ *
+ */
+public class TrustedChannelResolver implements Configurable {
+  Configuration conf;
+
+  /**
+   * Returns an instance of TrustedChannelResolver.
+   * Looks up the configuration to see if there is custom class specified.
+   * @param conf
+   * @return TrustedChannelResolver
+   */
+  public static TrustedChannelResolver getInstance(Configuration conf) {
+    Class<? extends TrustedChannelResolver> clazz =
+      conf.getClass(
+          HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
+          TrustedChannelResolver.class, TrustedChannelResolver.class);
+    return ReflectionUtils.newInstance(clazz, conf);
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Return boolean value indicating whether a channel is trusted or not
+   * from a client's perspective.
+   * @return true if the channel is trusted and false otherwise.
+   */
+  public boolean isTrusted() {
+    return false;
+  }
+
+
+  /**
+   * Identify boolean value indicating whether a channel is trusted or not.
+   * @param peerAddress address of the peer
+   * @return true if the channel is trusted and false otherwise.
+   */
+  public boolean isTrusted(InetAddress peerAddress) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java
new file mode 100644
index 0000000..959cba0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.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.protocol.datatransfer.sasl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+
+/**
+ * Creates a new {@link DataEncryptionKey} on demand.
+ */
+@InterfaceAudience.Private
+public interface DataEncryptionKeyFactory {
+
+  /**
+   * Creates a new DataEncryptionKey.
+   *
+   * @return DataEncryptionKey newly created
+   * @throws IOException for any error
+   */
+  DataEncryptionKey newDataEncryptionKey() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
new file mode 100644
index 0000000..256caff
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
@@ -0,0 +1,519 @@
+/**
+ * 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.protocol.datatransfer.sasl;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.security.sasl.Sasl;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherOption;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoInputStream;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto.DataTransferEncryptorStatus;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherOptionProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.security.SaslPropertiesResolver;
+import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.net.InetAddresses;
+import com.google.protobuf.ByteString;
+
+/**
+ * Utility methods implementing SASL negotiation for DataTransferProtocol.
+ */
+@InterfaceAudience.Private
+public final class DataTransferSaslUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    DataTransferSaslUtil.class);
+
+  /**
+   * Delimiter for the three-part SASL username string.
+   */
+  public static final String NAME_DELIMITER = " ";
+
+  /**
+   * Sent by clients and validated by servers. We use a number that's unlikely
+   * to ever be sent as the value of the DATA_TRANSFER_VERSION.
+   */
+  public static final int SASL_TRANSFER_MAGIC_NUMBER = 0xDEADBEEF;
+
+  /**
+   * Checks that SASL negotiation has completed for the given participant, and
+   * the negotiated quality of protection is included in the given SASL
+   * properties and therefore acceptable.
+   *
+   * @param sasl participant to check
+   * @param saslProps properties of SASL negotiation
+   * @throws IOException for any error
+   */
+  public static void checkSaslComplete(SaslParticipant sasl,
+      Map<String, String> saslProps) throws IOException {
+    if (!sasl.isComplete()) {
+      throw new IOException("Failed to complete SASL handshake");
+    }
+    Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
+      saslProps.get(Sasl.QOP).split(",")));
+    String negotiatedQop = sasl.getNegotiatedQop();
+    LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
+      requestedQop, negotiatedQop);
+    if (!requestedQop.contains(negotiatedQop)) {
+      throw new IOException(String.format("SASL handshake completed, but " +
+        "channel does not have acceptable quality of protection, " +
+        "requested = %s, negotiated = %s", requestedQop, negotiatedQop));
+    }
+  }
+  
+  /**
+   * Check whether requested SASL Qop contains privacy.
+   * 
+   * @param saslProps properties of SASL negotiation
+   * @return boolean true if privacy exists
+   */
+  public static boolean requestedQopContainsPrivacy(
+      Map<String, String> saslProps) {
+    Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
+        saslProps.get(Sasl.QOP).split(",")));
+    return requestedQop.contains("auth-conf");
+  }
+
+  /**
+   * Creates SASL properties required for an encrypted SASL negotiation.
+   *
+   * @param encryptionAlgorithm to use for SASL negotation
+   * @return properties of encrypted SASL negotiation
+   */
+  public static Map<String, String> createSaslPropertiesForEncryption(
+      String encryptionAlgorithm) {
+    Map<String, String> saslProps = Maps.newHashMapWithExpectedSize(3);
+    saslProps.put(Sasl.QOP, QualityOfProtection.PRIVACY.getSaslQop());
+    saslProps.put(Sasl.SERVER_AUTH, "true");
+    saslProps.put("com.sun.security.sasl.digest.cipher", encryptionAlgorithm);
+    return saslProps;
+  }
+
+  /**
+   * For an encrypted SASL negotiation, encodes an encryption key to a SASL
+   * password.
+   *
+   * @param encryptionKey to encode
+   * @return key encoded as SASL password
+   */
+  public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
+    return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
+      .toCharArray();
+  }
+
+  /**
+   * Returns InetAddress from peer.  The getRemoteAddressString has the form
+   * [host][/ip-address]:port.  The host may be missing.  The IP address (and
+   * preceding '/') may be missing.  The port preceded by ':' is always present.
+   *
+   * @param peer
+   * @return InetAddress from peer
+   */
+  public static InetAddress getPeerAddress(Peer peer) {
+    String remoteAddr = peer.getRemoteAddressString().split(":")[0];
+    int slashIdx = remoteAddr.indexOf('/');
+    return InetAddresses.forString(slashIdx != -1 ?
+        remoteAddr.substring(slashIdx + 1, remoteAddr.length()) :
+        remoteAddr);
+  }
+
+  /**
+   * Creates a SaslPropertiesResolver from the given configuration.  This method
+   * works by cloning the configuration, translating configuration properties
+   * specific to DataTransferProtocol to what SaslPropertiesResolver expects,
+   * and then delegating to SaslPropertiesResolver for initialization.  This
+   * method returns null if SASL protection has not been configured for
+   * DataTransferProtocol.
+   *
+   * @param conf configuration to read
+   * @return SaslPropertiesResolver for DataTransferProtocol, or null if not
+   *   configured
+   */
+  public static SaslPropertiesResolver getSaslPropertiesResolver(
+      Configuration conf) {
+    String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
+    if (qops == null || qops.isEmpty()) {
+      LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
+        "QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
+      return null;
+    }
+    Configuration saslPropsResolverConf = new Configuration(conf);
+    saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
+    Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
+      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+      SaslPropertiesResolver.class, SaslPropertiesResolver.class);
+    resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
+      resolverClass, SaslPropertiesResolver.class);
+    saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+      resolverClass, SaslPropertiesResolver.class);
+    SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
+      saslPropsResolverConf);
+    LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
+      "QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops, 
+      DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
+    return resolver;
+  }
+
+  /**
+   * Reads a SASL negotiation message.
+   *
+   * @param in stream to read
+   * @return bytes of SASL negotiation messsage
+   * @throws IOException for any error
+   */
+  public static byte[] readSaslMessage(InputStream in) throws IOException {
+    DataTransferEncryptorMessageProto proto =
+        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
+    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
+      throw new InvalidEncryptionKeyException(proto.getMessage());
+    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
+      throw new IOException(proto.getMessage());
+    } else {
+      return proto.getPayload().toByteArray();
+    }
+  }
+  
+  /**
+   * Reads a SASL negotiation message and negotiation cipher options. 
+   * 
+   * @param in stream to read
+   * @param cipherOptions list to store negotiation cipher options
+   * @return byte[] SASL negotiation message
+   * @throws IOException for any error
+   */
+  public static byte[] readSaslMessageAndNegotiationCipherOptions(
+      InputStream in, List<CipherOption> cipherOptions) throws IOException {
+    DataTransferEncryptorMessageProto proto =
+        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
+    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
+      throw new InvalidEncryptionKeyException(proto.getMessage());
+    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
+      throw new IOException(proto.getMessage());
+    } else {
+      List<CipherOptionProto> optionProtos = proto.getCipherOptionList();
+      if (optionProtos != null) {
+        for (CipherOptionProto optionProto : optionProtos) {
+          cipherOptions.add(PBHelperClient.convert(optionProto));
+        }
+      }
+      return proto.getPayload().toByteArray();
+    }
+  }
+  
+  /**
+   * Negotiate a cipher option which server supports.
+   * 
+   * @param conf the configuration
+   * @param options the cipher options which client supports
+   * @return CipherOption negotiated cipher option
+   */
+  public static CipherOption negotiateCipherOption(Configuration conf,
+      List<CipherOption> options) throws IOException {
+    // Negotiate cipher suites if configured.  Currently, the only supported
+    // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
+    // values for future expansion.
+    String cipherSuites = conf.get(DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
+    if (cipherSuites == null || cipherSuites.isEmpty()) {
+      return null;
+    }
+    if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
+      throw new IOException(String.format("Invalid cipher suite, %s=%s",
+          DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
+    }
+    if (options != null) {
+      for (CipherOption option : options) {
+        CipherSuite suite = option.getCipherSuite();
+        if (suite == CipherSuite.AES_CTR_NOPADDING) {
+          int keyLen = conf.getInt(
+              DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY,
+              DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT) / 8;
+          CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+          byte[] inKey = new byte[keyLen];
+          byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
+          byte[] outKey = new byte[keyLen];
+          byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
+          codec.generateSecureRandom(inKey);
+          codec.generateSecureRandom(inIv);
+          codec.generateSecureRandom(outKey);
+          codec.generateSecureRandom(outIv);
+          return new CipherOption(suite, inKey, inIv, outKey, outIv);
+        }
+      }
+    }
+    return null;
+  }
+  
+  /**
+   * Send SASL message and negotiated cipher option to client.
+   * 
+   * @param out stream to receive message
+   * @param payload to send
+   * @param option negotiated cipher option
+   * @throws IOException for any error
+   */
+  public static void sendSaslMessageAndNegotiatedCipherOption(
+      OutputStream out, byte[] payload, CipherOption option) 
+          throws IOException {
+    DataTransferEncryptorMessageProto.Builder builder =
+        DataTransferEncryptorMessageProto.newBuilder();
+    
+    builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
+    if (payload != null) {
+      builder.setPayload(ByteString.copyFrom(payload));
+    }
+    if (option != null) {
+      builder.addCipherOption(PBHelperClient.convert(option));
+    }
+    
+    DataTransferEncryptorMessageProto proto = builder.build();
+    proto.writeDelimitedTo(out);
+    out.flush();
+  }
+  
+  /**
+   * Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
+   * and {@link org.apache.hadoop.crypto.CryptoOutputStream}
+   * 
+   * @param conf the configuration
+   * @param cipherOption negotiated cipher option
+   * @param out underlying output stream
+   * @param in underlying input stream
+   * @param isServer is server side
+   * @return IOStreamPair the stream pair
+   * @throws IOException for any error
+   */
+  public static IOStreamPair createStreamPair(Configuration conf,
+      CipherOption cipherOption, OutputStream out, InputStream in, 
+      boolean isServer) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
+          "CryptoOutputStream.");
+    }
+    CryptoCodec codec = CryptoCodec.getInstance(conf, 
+        cipherOption.getCipherSuite());
+    byte[] inKey = cipherOption.getInKey();
+    byte[] inIv = cipherOption.getInIv();
+    byte[] outKey = cipherOption.getOutKey();
+    byte[] outIv = cipherOption.getOutIv();
+    InputStream cIn = new CryptoInputStream(in, codec, 
+        isServer ? inKey : outKey, isServer ? inIv : outIv);
+    OutputStream cOut = new CryptoOutputStream(out, codec, 
+        isServer ? outKey : inKey, isServer ? outIv : inIv);
+    return new IOStreamPair(cIn, cOut);
+  }
+
+  /**
+   * Sends a SASL negotiation message indicating an error.
+   *
+   * @param out stream to receive message
+   * @param message to send
+   * @throws IOException for any error
+   */
+  public static void sendGenericSaslErrorMessage(OutputStream out,
+      String message) throws IOException {
+    sendSaslMessage(out, DataTransferEncryptorStatus.ERROR, null, message);
+  }
+
+  /**
+   * Sends a SASL negotiation message.
+   *
+   * @param out stream to receive message
+   * @param payload to send
+   * @throws IOException for any error
+   */
+  public static void sendSaslMessage(OutputStream out, byte[] payload)
+      throws IOException {
+    sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
+  }
+  
+  /**
+   * Send a SASL negotiation message and negotiation cipher options to server.
+   * 
+   * @param out stream to receive message
+   * @param payload to send
+   * @param options cipher options to negotiate
+   * @throws IOException for any error
+   */
+  public static void sendSaslMessageAndNegotiationCipherOptions(
+      OutputStream out, byte[] payload, List<CipherOption> options)
+          throws IOException {
+    DataTransferEncryptorMessageProto.Builder builder =
+        DataTransferEncryptorMessageProto.newBuilder();
+    
+    builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
+    if (payload != null) {
+      builder.setPayload(ByteString.copyFrom(payload));
+    }
+    if (options != null) {
+      builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options));
+    }
+    
+    DataTransferEncryptorMessageProto proto = builder.build();
+    proto.writeDelimitedTo(out);
+    out.flush();
+  }
+  
+  /**
+   * Read SASL message and negotiated cipher option from server.
+   * 
+   * @param in stream to read
+   * @return SaslResponseWithNegotiatedCipherOption SASL message and 
+   * negotiated cipher option
+   * @throws IOException for any error
+   */
+  public static SaslResponseWithNegotiatedCipherOption
+      readSaslMessageAndNegotiatedCipherOption(InputStream in)
+          throws IOException {
+    DataTransferEncryptorMessageProto proto =
+        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
+    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
+      throw new InvalidEncryptionKeyException(proto.getMessage());
+    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
+      throw new IOException(proto.getMessage());
+    } else {
+      byte[] response = proto.getPayload().toByteArray();
+      List<CipherOption> options = PBHelperClient.convertCipherOptionProtos(
+          proto.getCipherOptionList());
+      CipherOption option = null;
+      if (options != null && !options.isEmpty()) {
+        option = options.get(0);
+      }
+      return new SaslResponseWithNegotiatedCipherOption(response, option);
+    }
+  }
+  
+  /**
+   * Encrypt the key and iv of the negotiated cipher option.
+   * 
+   * @param option negotiated cipher option
+   * @param sasl SASL participant representing server
+   * @return CipherOption negotiated cipher option which contains the 
+   * encrypted key and iv
+   * @throws IOException for any error
+   */
+  public static CipherOption wrap(CipherOption option, SaslParticipant sasl) 
+      throws IOException {
+    if (option != null) {
+      byte[] inKey = option.getInKey();
+      if (inKey != null) {
+        inKey = sasl.wrap(inKey, 0, inKey.length);
+      }
+      byte[] outKey = option.getOutKey();
+      if (outKey != null) {
+        outKey = sasl.wrap(outKey, 0, outKey.length);
+      }
+      return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
+          outKey, option.getOutIv());
+    }
+    
+    return null;
+  }
+  
+  /**
+   * Decrypt the key and iv of the negotiated cipher option.
+   * 
+   * @param option negotiated cipher option
+   * @param sasl SASL participant representing client
+   * @return CipherOption negotiated cipher option which contains the 
+   * decrypted key and iv
+   * @throws IOException for any error
+   */
+  public static CipherOption unwrap(CipherOption option, SaslParticipant sasl)
+      throws IOException {
+    if (option != null) {
+      byte[] inKey = option.getInKey();
+      if (inKey != null) {
+        inKey = sasl.unwrap(inKey, 0, inKey.length);
+      }
+      byte[] outKey = option.getOutKey();
+      if (outKey != null) {
+        outKey = sasl.unwrap(outKey, 0, outKey.length);
+      }
+      return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
+          outKey, option.getOutIv());
+    }
+    
+    return null;
+  }
+
+  /**
+   * Sends a SASL negotiation message.
+   *
+   * @param out stream to receive message
+   * @param status negotiation status
+   * @param payload to send
+   * @param message to send
+   * @throws IOException for any error
+   */
+  public static void sendSaslMessage(OutputStream out,
+      DataTransferEncryptorStatus status, byte[] payload, String message)
+          throws IOException {
+    DataTransferEncryptorMessageProto.Builder builder =
+        DataTransferEncryptorMessageProto.newBuilder();
+    
+    builder.setStatus(status);
+    if (payload != null) {
+      builder.setPayload(ByteString.copyFrom(payload));
+    }
+    if (message != null) {
+      builder.setMessage(message);
+    }
+    
+    DataTransferEncryptorMessageProto proto = builder.build();
+    proto.writeDelimitedTo(out);
+    out.flush();
+  }
+
+  /**
+   * There is no reason to instantiate this class.
+   */
+  private DataTransferSaslUtil() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
new file mode 100644
index 0000000..913203c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
@@ -0,0 +1,498 @@
+/**
+ * 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.protocol.datatransfer.sasl;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
+import static org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.*;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.RealmChoiceCallback;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherOption;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.hdfs.net.EncryptedPeer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.security.SaslPropertiesResolver;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+
+/**
+ * Negotiates SASL for DataTransferProtocol on behalf of a client.  There are
+ * two possible supported variants of SASL negotiation: either a general-purpose
+ * negotiation supporting any quality of protection, or a specialized
+ * negotiation that enforces privacy as the quality of protection using a
+ * cryptographically strong encryption key.
+ *
+ * This class is used in both the HDFS client and the DataNode.  The DataNode
+ * needs it, because it acts as a client to other DataNodes during write
+ * pipelines and block transfers.
+ */
+@InterfaceAudience.Private
+public class SaslDataTransferClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    SaslDataTransferClient.class);
+
+  private final Configuration conf;
+  private final AtomicBoolean fallbackToSimpleAuth;
+  private final SaslPropertiesResolver saslPropsResolver;
+  private final TrustedChannelResolver trustedChannelResolver;
+
+  /**
+   * Creates a new SaslDataTransferClient.  This constructor is used in cases
+   * where it is not relevant to track if a secure client did a fallback to
+   * simple auth.  For intra-cluster connections between data nodes in the same
+   * cluster, we can assume that all run under the same security configuration.
+   *
+   * @param conf the configuration
+   * @param saslPropsResolver for determining properties of SASL negotiation
+   * @param trustedChannelResolver for identifying trusted connections that do
+   *   not require SASL negotiation
+   */
+  public SaslDataTransferClient(Configuration conf, 
+      SaslPropertiesResolver saslPropsResolver,
+      TrustedChannelResolver trustedChannelResolver) {
+    this(conf, saslPropsResolver, trustedChannelResolver, null);
+  }
+
+  /**
+   * Creates a new SaslDataTransferClient.
+   *
+   * @param conf the configuration
+   * @param saslPropsResolver for determining properties of SASL negotiation
+   * @param trustedChannelResolver for identifying trusted connections that do
+   *   not require SASL negotiation
+   * @param fallbackToSimpleAuth checked on each attempt at general SASL
+   *   handshake, if true forces use of simple auth
+   */
+  public SaslDataTransferClient(Configuration conf, 
+      SaslPropertiesResolver saslPropsResolver,
+      TrustedChannelResolver trustedChannelResolver,
+      AtomicBoolean fallbackToSimpleAuth) {
+    this.conf = conf;
+    this.fallbackToSimpleAuth = fallbackToSimpleAuth;
+    this.saslPropsResolver = saslPropsResolver;
+    this.trustedChannelResolver = trustedChannelResolver;
+  }
+
+  /**
+   * Sends client SASL negotiation for a newly allocated socket if required.
+   *
+   * @param socket connection socket
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param encryptionKeyFactory for creation of an encryption key
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  public IOStreamPair newSocketSend(Socket socket, OutputStream underlyingOut,
+      InputStream underlyingIn, DataEncryptionKeyFactory encryptionKeyFactory,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    // The encryption key factory only returns a key if encryption is enabled.
+    DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
+      encryptionKeyFactory.newDataEncryptionKey() : null;
+    IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
+      underlyingIn, encryptionKey, accessToken, datanodeId);
+    return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
+  }
+
+  /**
+   * Sends client SASL negotiation for a peer if required.
+   *
+   * @param peer connection peer
+   * @param encryptionKeyFactory for creation of an encryption key
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  public Peer peerSend(Peer peer, DataEncryptionKeyFactory encryptionKeyFactory,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
+      peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
+      accessToken, datanodeId);
+    // TODO: Consider renaming EncryptedPeer to SaslPeer.
+    return ios != null ? new EncryptedPeer(peer, ios) : peer;
+  }
+
+  /**
+   * Sends client SASL negotiation for a socket if required.
+   *
+   * @param socket connection socket
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param encryptionKeyFactory for creation of an encryption key
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  public IOStreamPair socketSend(Socket socket, OutputStream underlyingOut,
+      InputStream underlyingIn, DataEncryptionKeyFactory encryptionKeyFactory,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
+      underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
+    return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
+  }
+
+  /**
+   * Checks if an address is already trusted and then sends client SASL
+   * negotiation if required.
+   *
+   * @param addr connection address
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param encryptionKeyFactory for creation of an encryption key
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  private IOStreamPair checkTrustAndSend(InetAddress addr,
+      OutputStream underlyingOut, InputStream underlyingIn,
+      DataEncryptionKeyFactory encryptionKeyFactory,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    if (!trustedChannelResolver.isTrusted() &&
+        !trustedChannelResolver.isTrusted(addr)) {
+      // The encryption key factory only returns a key if encryption is enabled.
+      DataEncryptionKey encryptionKey =
+        encryptionKeyFactory.newDataEncryptionKey();
+      return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
+        datanodeId);
+    } else {
+      LOG.debug(
+        "SASL client skipping handshake on trusted connection for addr = {}, "
+        + "datanodeId = {}", addr, datanodeId);
+      return null;
+    }
+  }
+
+  /**
+   * Sends client SASL negotiation if required.  Determines the correct type of
+   * SASL handshake based on configuration.
+   *
+   * @param addr connection address
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param encryptionKey for an encrypted SASL handshake
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  private IOStreamPair send(InetAddress addr, OutputStream underlyingOut,
+      InputStream underlyingIn, DataEncryptionKey encryptionKey,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    if (encryptionKey != null) {
+      LOG.debug(
+        "SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
+        addr, datanodeId);
+      return getEncryptedStreams(underlyingOut, underlyingIn,
+        encryptionKey);
+    } else if (!UserGroupInformation.isSecurityEnabled()) {
+      LOG.debug(
+        "SASL client skipping handshake in unsecured configuration for "
+        + "addr = {}, datanodeId = {}", addr, datanodeId);
+      return null;
+    } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
+      LOG.debug(
+        "SASL client skipping handshake in secured configuration with "
+        + "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
+      return null;
+    } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
+      LOG.debug(
+        "SASL client skipping handshake in secured configuration with "
+        + "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
+      return null;
+    } else if (saslPropsResolver != null) {
+      LOG.debug(
+        "SASL client doing general handshake for addr = {}, datanodeId = {}",
+        addr, datanodeId);
+      return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
+        datanodeId);
+    } else {
+      // It's a secured cluster using non-privileged ports, but no SASL.  The
+      // only way this can happen is if the DataNode has
+      // ignore.secure.ports.for.testing configured, so this is a rare edge case.
+      LOG.debug(
+        "SASL client skipping handshake in secured configuration with no SASL "
+        + "protection configured for addr = {}, datanodeId = {}",
+        addr, datanodeId);
+      return null;
+    }
+  }
+
+  /**
+   * Sends client SASL negotiation for specialized encrypted handshake.
+   *
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param encryptionKey for an encrypted SASL handshake
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  private IOStreamPair getEncryptedStreams(OutputStream underlyingOut,
+      InputStream underlyingIn, DataEncryptionKey encryptionKey)
+      throws IOException {
+    Map<String, String> saslProps = createSaslPropertiesForEncryption(
+      encryptionKey.encryptionAlgorithm);
+
+    LOG.debug("Client using encryption algorithm {}",
+      encryptionKey.encryptionAlgorithm);
+
+    String userName = getUserNameFromEncryptionKey(encryptionKey);
+    char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
+    CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
+      password);
+    return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
+      callbackHandler);
+  }
+
+  /**
+   * The SASL username for an encrypted handshake consists of the keyId,
+   * blockPoolId, and nonce with the first two encoded as Strings, and the third
+   * encoded using Base64. The fields are each separated by a single space.
+   * 
+   * @param encryptionKey the encryption key to encode as a SASL username.
+   * @return encoded username containing keyId, blockPoolId, and nonce
+   */
+  private static String getUserNameFromEncryptionKey(
+      DataEncryptionKey encryptionKey) {
+    return encryptionKey.keyId + NAME_DELIMITER +
+        encryptionKey.blockPoolId + NAME_DELIMITER +
+        new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
+  }
+
+  /**
+   * Sets user name and password when asked by the client-side SASL object.
+   */
+  private static final class SaslClientCallbackHandler
+      implements CallbackHandler {
+
+    private final char[] password;
+    private final String userName;
+
+    /**
+     * Creates a new SaslClientCallbackHandler.
+     *
+     * @param userName SASL user name
+     * @Param password SASL password
+     */
+    public SaslClientCallbackHandler(String userName, char[] password) {
+      this.password = password;
+      this.userName = userName;
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException,
+        UnsupportedCallbackException {
+      NameCallback nc = null;
+      PasswordCallback pc = null;
+      RealmCallback rc = null;
+      for (Callback callback : callbacks) {
+        if (callback instanceof RealmChoiceCallback) {
+          continue;
+        } else if (callback instanceof NameCallback) {
+          nc = (NameCallback) callback;
+        } else if (callback instanceof PasswordCallback) {
+          pc = (PasswordCallback) callback;
+        } else if (callback instanceof RealmCallback) {
+          rc = (RealmCallback) callback;
+        } else {
+          throw new UnsupportedCallbackException(callback,
+              "Unrecognized SASL client callback");
+        }
+      }
+      if (nc != null) {
+        nc.setName(userName);
+      }
+      if (pc != null) {
+        pc.setPassword(password);
+      }
+      if (rc != null) {
+        rc.setText(rc.getDefaultText());
+      }
+    }
+  }
+
+  /**
+   * Sends client SASL negotiation for general-purpose handshake.
+   *
+   * @param addr connection address
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param accessToken connection block access token
+   * @param datanodeId ID of destination DataNode
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  private IOStreamPair getSaslStreams(InetAddress addr,
+      OutputStream underlyingOut, InputStream underlyingIn,
+      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      throws IOException {
+    Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
+
+    String userName = buildUserName(accessToken);
+    char[] password = buildClientPassword(accessToken);
+    CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
+      password);
+    return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
+      callbackHandler);
+  }
+
+  /**
+   * Builds the client's user name for the general-purpose handshake, consisting
+   * of the base64-encoded serialized block access token identifier.  Note that
+   * this includes only the token identifier, not the token itself, which would
+   * include the password.  The password is a shared secret, and we must not
+   * write it on the network during the SASL authentication exchange.
+   *
+   * @param blockToken for block access
+   * @return SASL user name
+   */
+  private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
+    return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
+      Charsets.UTF_8);
+  }
+
+  /**
+   * Calculates the password on the client side for the general-purpose
+   * handshake.  The password consists of the block access token's password.
+   *
+   * @param blockToken for block access
+   * @return SASL password
+   */    
+  private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
+    return new String(Base64.encodeBase64(blockToken.getPassword(), false),
+      Charsets.UTF_8).toCharArray();
+  }
+
+  /**
+   * This method actually executes the client-side SASL handshake.
+   *
+   * @param underlyingOut connection output stream
+   * @param underlyingIn connection input stream
+   * @param userName SASL user name
+   * @param saslProps properties of SASL negotiation
+   * @param callbackHandler for responding to SASL callbacks
+   * @return new pair of streams, wrapped after SASL negotiation
+   * @throws IOException for any error
+   */
+  private IOStreamPair doSaslHandshake(OutputStream underlyingOut,
+      InputStream underlyingIn, String userName, Map<String, String> saslProps,
+      CallbackHandler callbackHandler) throws IOException {
+
+    DataOutputStream out = new DataOutputStream(underlyingOut);
+    DataInputStream in = new DataInputStream(underlyingIn);
+
+    SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
+      saslProps, callbackHandler);
+
+    out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
+    out.flush();
+
+    try {
+      // Start of handshake - "initial response" in SASL terminology.
+      sendSaslMessage(out, new byte[0]);
+
+      // step 1
+      byte[] remoteResponse = readSaslMessage(in);
+      byte[] localResponse = sasl.evaluateChallengeOrResponse(remoteResponse);
+      List<CipherOption> cipherOptions = null;
+      if (requestedQopContainsPrivacy(saslProps)) {
+        // Negotiate cipher suites if configured.  Currently, the only supported
+        // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
+        // values for future expansion.
+        String cipherSuites = conf.get(
+            DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
+        if (cipherSuites != null && !cipherSuites.isEmpty()) {
+          if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
+            throw new IOException(String.format("Invalid cipher suite, %s=%s",
+                DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
+          }
+          CipherOption option = new CipherOption(CipherSuite.AES_CTR_NOPADDING);
+          cipherOptions = Lists.newArrayListWithCapacity(1);
+          cipherOptions.add(option);
+        }
+      }
+      sendSaslMessageAndNegotiationCipherOptions(out, localResponse, 
+          cipherOptions);
+
+      // step 2 (client-side only)
+      SaslResponseWithNegotiatedCipherOption response = 
+          readSaslMessageAndNegotiatedCipherOption(in);
+      localResponse = sasl.evaluateChallengeOrResponse(response.payload);
+      assert localResponse == null;
+
+      // SASL handshake is complete
+      checkSaslComplete(sasl, saslProps);
+
+      CipherOption cipherOption = null;
+      if (sasl.isNegotiatedQopPrivacy()) {
+        // Unwrap the negotiated cipher option
+        cipherOption = unwrap(response.cipherOption, sasl);
+      }
+
+      // If negotiated cipher option is not null, we will use it to create 
+      // stream pair.
+      return cipherOption != null ? createStreamPair(
+          conf, cipherOption, underlyingOut, underlyingIn, false) : 
+            sasl.createStreamPair(out, in);
+    } catch (IOException ioe) {
+      sendGenericSaslErrorMessage(out, ioe.getMessage());
+      throw ioe;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
new file mode 100644
index 0000000..f14a075
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
@@ -0,0 +1,210 @@
+/**
+ * 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.protocol.datatransfer.sasl;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.Map;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.security.SaslInputStream;
+import org.apache.hadoop.security.SaslOutputStream;
+
+/**
+ * Strongly inspired by Thrift's TSaslTransport class.
+ *
+ * Used to abstract over the <code>SaslServer</code> and
+ * <code>SaslClient</code> classes, which share a lot of their interface, but
+ * unfortunately don't share a common superclass.
+ */
+@InterfaceAudience.Private
+class SaslParticipant {
+
+  // This has to be set as part of the SASL spec, but it don't matter for
+  // our purposes, but may not be empty. It's sent over the wire, so use
+  // a short string.
+  private static final String SERVER_NAME = "0";
+  private static final String PROTOCOL = "hdfs";
+  private static final String MECHANISM = "DIGEST-MD5";
+
+  // One of these will always be null.
+  private final SaslServer saslServer;
+  private final SaslClient saslClient;
+
+  /**
+   * Creates a SaslParticipant wrapping a SaslServer.
+   *
+   * @param saslProps properties of SASL negotiation
+   * @param callbackHandler for handling all SASL callbacks
+   * @return SaslParticipant wrapping SaslServer
+   * @throws SaslException for any error
+   */
+  public static SaslParticipant createServerSaslParticipant(
+      Map<String, String> saslProps, CallbackHandler callbackHandler)
+      throws SaslException {
+    return new SaslParticipant(Sasl.createSaslServer(MECHANISM,
+      PROTOCOL, SERVER_NAME, saslProps, callbackHandler));
+  }
+
+  /**
+   * Creates a SaslParticipant wrapping a SaslClient.
+   *
+   * @param userName SASL user name
+   * @param saslProps properties of SASL negotiation
+   * @param callbackHandler for handling all SASL callbacks
+   * @return SaslParticipant wrapping SaslClient
+   * @throws SaslException for any error
+   */
+  public static SaslParticipant createClientSaslParticipant(String userName,
+      Map<String, String> saslProps, CallbackHandler callbackHandler)
+      throws SaslException {
+    return new SaslParticipant(Sasl.createSaslClient(new String[] { MECHANISM },
+      userName, PROTOCOL, SERVER_NAME, saslProps, callbackHandler));
+  }
+
+  /**
+   * Private constructor wrapping a SaslServer.
+   *
+   * @param saslServer to wrap
+   */
+  private SaslParticipant(SaslServer saslServer) {
+    this.saslServer = saslServer;
+    this.saslClient = null;
+  }
+
+  /**
+   * Private constructor wrapping a SaslClient.
+   *
+   * @param saslClient to wrap
+   */
+  private SaslParticipant(SaslClient saslClient) {
+    this.saslServer = null;
+    this.saslClient = saslClient;
+  }
+
+  /**
+   * @see {@link SaslServer#evaluateResponse}
+   * @see {@link SaslClient#evaluateChallenge}
+   */
+  public byte[] evaluateChallengeOrResponse(byte[] challengeOrResponse)
+      throws SaslException {
+    if (saslClient != null) {
+      return saslClient.evaluateChallenge(challengeOrResponse);
+    } else {
+      return saslServer.evaluateResponse(challengeOrResponse);
+    }
+  }
+
+  /**
+   * After successful SASL negotation, returns the negotiated quality of
+   * protection.
+   *
+   * @return negotiated quality of protection
+   */
+  public String getNegotiatedQop() {
+    if (saslClient != null) {
+      return (String) saslClient.getNegotiatedProperty(Sasl.QOP);
+    } else {
+      return (String) saslServer.getNegotiatedProperty(Sasl.QOP);
+    }
+  }
+  
+  /**
+   * After successful SASL negotiation, returns whether it's QOP privacy
+   * 
+   * @return boolean whether it's QOP privacy
+   */
+  public boolean isNegotiatedQopPrivacy() {
+    String qop = getNegotiatedQop();
+    return qop != null && "auth-conf".equalsIgnoreCase(qop);
+  }
+  
+  /**
+   * Wraps a byte array.
+   * 
+   * @param bytes The array containing the bytes to wrap.
+   * @param off The starting position at the array
+   * @param len The number of bytes to wrap
+   * @return byte[] wrapped bytes
+   * @throws SaslException if the bytes cannot be successfully wrapped
+   */
+  public byte[] wrap(byte[] bytes, int off, int len) throws SaslException {
+    if (saslClient != null) {
+      return saslClient.wrap(bytes, off, len);
+    } else {
+      return saslServer.wrap(bytes, off, len);
+    }
+  }
+  
+  /**
+   * Unwraps a byte array.
+   * 
+   * @param bytes The array containing the bytes to unwrap.
+   * @param off The starting position at the array
+   * @param len The number of bytes to unwrap
+   * @return byte[] unwrapped bytes
+   * @throws SaslException if the bytes cannot be successfully unwrapped
+   */
+  public byte[] unwrap(byte[] bytes, int off, int len) throws SaslException {
+    if (saslClient != null) {
+      return saslClient.unwrap(bytes, off, len);
+    } else {
+      return saslServer.unwrap(bytes, off, len);
+    }
+  }
+
+  /**
+   * Returns true if SASL negotiation is complete.
+   *
+   * @return true if SASL negotiation is complete
+   */
+  public boolean isComplete() {
+    if (saslClient != null) {
+      return saslClient.isComplete();
+    } else {
+      return saslServer.isComplete();
+    }
+  }
+
+  /**
+   * Return some input/output streams that may henceforth have their
+   * communication encrypted, depending on the negotiated quality of protection.
+   *
+   * @param out output stream to wrap
+   * @param in input stream to wrap
+   * @return IOStreamPair wrapping the streams
+   */
+  public IOStreamPair createStreamPair(DataOutputStream out,
+      DataInputStream in) {
+    if (saslClient != null) {
+      return new IOStreamPair(
+          new SaslInputStream(in, saslClient),
+          new SaslOutputStream(out, saslClient));
+    } else {
+      return new IOStreamPair(
+          new SaslInputStream(in, saslServer),
+          new SaslOutputStream(out, saslServer));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
new file mode 100644
index 0000000..f69441b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.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.protocol.datatransfer.sasl;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.CipherOption;
+
+@InterfaceAudience.Private
+public class SaslResponseWithNegotiatedCipherOption {
+  final byte[] payload;
+  final CipherOption cipherOption;
+  
+  public SaslResponseWithNegotiatedCipherOption(byte[] payload, 
+      CipherOption cipherOption) {
+    this.payload = payload;
+    this.cipherOption = cipherOption;
+  }
+}
\ No newline at end of file


[04/50] [abbrv] hadoop git commit: HDFS-8990. Move RemoteBlockReader to hdfs-client module. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
deleted file mode 100644
index 2a77cb6..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ /dev/null
@@ -1,477 +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.hdfs;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.util.EnumSet;
-import java.util.UUID;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * This is a wrapper around connection to datanode
- * and understands checksum, offset etc.
- *
- * Terminology:
- * <dl>
- * <dt>block</dt>
- *   <dd>The hdfs block, typically large (~64MB).
- *   </dd>
- * <dt>chunk</dt>
- *   <dd>A block is divided into chunks, each comes with a checksum.
- *       We want transfers to be chunk-aligned, to be able to
- *       verify checksums.
- *   </dd>
- * <dt>packet</dt>
- *   <dd>A grouping of chunks used for transport. It contains a
- *       header, followed by checksum data, followed by real data.
- *   </dd>
- * </dl>
- * Please see DataNode for the RPC specification.
- *
- * This is a new implementation introduced in Hadoop 0.23 which
- * is more efficient and simpler than the older BlockReader
- * implementation. It should be renamed to RemoteBlockReader
- * once we are confident in it.
- */
-@InterfaceAudience.Private
-public class RemoteBlockReader2  implements BlockReader {
-
-  static final Log LOG = LogFactory.getLog(RemoteBlockReader2.class);
-  
-  final private Peer peer;
-  final private DatanodeID datanodeID;
-  final private PeerCache peerCache;
-  final private long blockId;
-  private final ReadableByteChannel in;
-  private DataChecksum checksum;
-  
-  private final PacketReceiver packetReceiver = new PacketReceiver(true);
-  private ByteBuffer curDataSlice = null;
-
-  /** offset in block of the last chunk received */
-  private long lastSeqNo = -1;
-
-  /** offset in block where reader wants to actually read */
-  private long startOffset;
-  private final String filename;
-
-  private final int bytesPerChecksum;
-  private final int checksumSize;
-
-  /**
-   * The total number of bytes we need to transfer from the DN.
-   * This is the amount that the user has requested plus some padding
-   * at the beginning so that the read can begin on a chunk boundary.
-   */
-  private long bytesNeededToFinish;
-
-  /**
-   * True if we are reading from a local DataNode.
-   */
-  private final boolean isLocal;
-
-  private final boolean verifyChecksum;
-
-  private boolean sentStatusCode = false;
-
-  @VisibleForTesting
-  public Peer getPeer() {
-    return peer;
-  }
-  
-  @Override
-  public synchronized int read(byte[] buf, int off, int len) 
-                               throws IOException {
-
-    UUID randomId = null;
-    if (LOG.isTraceEnabled()) {
-      randomId = UUID.randomUUID();
-      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
-        randomId.toString(), this.filename,
-        this.datanodeID.getHostName()));
-    }
-
-    if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-      TraceScope scope = Trace.startSpan(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
-      try {
-        readNextPacket();
-      } finally {
-        scope.close();
-      }
-    }
-
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Finishing read #" + randomId));
-    }
-
-    if (curDataSlice.remaining() == 0) {
-      // we're at EOF now
-      return -1;
-    }
-    
-    int nRead = Math.min(curDataSlice.remaining(), len);
-    curDataSlice.get(buf, off, nRead);
-    
-    return nRead;
-  }
-
-
-  @Override
-  public synchronized int read(ByteBuffer buf) throws IOException {
-    if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-      TraceScope scope = Trace.startSpan(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
-      try {
-        readNextPacket();
-      } finally {
-        scope.close();
-      }
-    }
-    if (curDataSlice.remaining() == 0) {
-      // we're at EOF now
-      return -1;
-    }
-
-    int nRead = Math.min(curDataSlice.remaining(), buf.remaining());
-    ByteBuffer writeSlice = curDataSlice.duplicate();
-    writeSlice.limit(writeSlice.position() + nRead);
-    buf.put(writeSlice);
-    curDataSlice.position(writeSlice.position());
-
-    return nRead;
-  }
-
-  private void readNextPacket() throws IOException {
-    //Read packet headers.
-    packetReceiver.receiveNextPacket(in);
-
-    PacketHeader curHeader = packetReceiver.getHeader();
-    curDataSlice = packetReceiver.getDataSlice();
-    assert curDataSlice.capacity() == curHeader.getDataLen();
-    
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("DFSClient readNextPacket got header " + curHeader);
-    }
-
-    // Sanity check the lengths
-    if (!curHeader.sanityCheck(lastSeqNo)) {
-         throw new IOException("BlockReader: error in packet header " +
-                               curHeader);
-    }
-    
-    if (curHeader.getDataLen() > 0) {
-      int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
-      int checksumsLen = chunks * checksumSize;
-
-      assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
-        "checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + 
-          " checksumsLen=" + checksumsLen;
-      
-      lastSeqNo = curHeader.getSeqno();
-      if (verifyChecksum && curDataSlice.remaining() > 0) {
-        // N.B.: the checksum error offset reported here is actually
-        // relative to the start of the block, not the start of the file.
-        // This is slightly misleading, but preserves the behavior from
-        // the older BlockReader.
-        checksum.verifyChunkedSums(curDataSlice,
-            packetReceiver.getChecksumSlice(),
-            filename, curHeader.getOffsetInBlock());
-      }
-      bytesNeededToFinish -= curHeader.getDataLen();
-    }    
-    
-    // First packet will include some data prior to the first byte
-    // the user requested. Skip it.
-    if (curHeader.getOffsetInBlock() < startOffset) {
-      int newPos = (int) (startOffset - curHeader.getOffsetInBlock());
-      curDataSlice.position(newPos);
-    }
-
-    // If we've now satisfied the whole client read, read one last packet
-    // header, which should be empty
-    if (bytesNeededToFinish <= 0) {
-      readTrailingEmptyPacket();
-      if (verifyChecksum) {
-        sendReadResult(Status.CHECKSUM_OK);
-      } else {
-        sendReadResult(Status.SUCCESS);
-      }
-    }
-  }
-  
-  @Override
-  public synchronized long skip(long n) throws IOException {
-    /* How can we make sure we don't throw a ChecksumException, at least
-     * in majority of the cases?. This one throws. */
-    long skipped = 0;
-    while (skipped < n) {
-      long needToSkip = n - skipped;
-      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-        readNextPacket();
-      }
-      if (curDataSlice.remaining() == 0) {
-        // we're at EOF now
-        break;
-      }
-
-      int skip = (int)Math.min(curDataSlice.remaining(), needToSkip);
-      curDataSlice.position(curDataSlice.position() + skip);
-      skipped += skip;
-    }
-    return skipped;
-  }
-
-  private void readTrailingEmptyPacket() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Reading empty packet at end of read");
-    }
-    
-    packetReceiver.receiveNextPacket(in);
-
-    PacketHeader trailer = packetReceiver.getHeader();
-    if (!trailer.isLastPacketInBlock() ||
-       trailer.getDataLen() != 0) {
-      throw new IOException("Expected empty end-of-read packet! Header: " +
-                            trailer);
-    }
-  }
-
-  protected RemoteBlockReader2(String file, String bpid, long blockId,
-      DataChecksum checksum, boolean verifyChecksum,
-      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
-      DatanodeID datanodeID, PeerCache peerCache) {
-    this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
-        createSocketAddr(datanodeID.getXferAddr()));
-    // Path is used only for printing block and file information in debug
-    this.peer = peer;
-    this.datanodeID = datanodeID;
-    this.in = peer.getInputStreamChannel();
-    this.checksum = checksum;
-    this.verifyChecksum = verifyChecksum;
-    this.startOffset = Math.max( startOffset, 0 );
-    this.filename = file;
-    this.peerCache = peerCache;
-    this.blockId = blockId;
-
-    // The total number of bytes that we need to transfer from the DN is
-    // the amount that the user wants (bytesToRead), plus the padding at
-    // the beginning in order to chunk-align. Note that the DN may elect
-    // to send more than this amount if the read starts/ends mid-chunk.
-    this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
-    bytesPerChecksum = this.checksum.getBytesPerChecksum();
-    checksumSize = this.checksum.getChecksumSize();
-  }
-
-
-  @Override
-  public synchronized void close() throws IOException {
-    packetReceiver.close();
-    startOffset = -1;
-    checksum = null;
-    if (peerCache != null && sentStatusCode) {
-      peerCache.put(datanodeID, peer);
-    } else {
-      peer.close();
-    }
-
-    // in will be closed when its Socket is closed.
-  }
-  
-  /**
-   * When the reader reaches end of the read, it sends a status response
-   * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
-   * closing our connection (which we will re-open), but won't affect
-   * data correctness.
-   */
-  void sendReadResult(Status statusCode) {
-    assert !sentStatusCode : "already sent status code to " + peer;
-    try {
-      writeReadResult(peer.getOutputStream(), statusCode);
-      sentStatusCode = true;
-    } catch (IOException e) {
-      // It's ok not to be able to send this. But something is probably wrong.
-      LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               peer.getRemoteAddressString() + ": " + e.getMessage());
-    }
-  }
-
-  /**
-   * Serialize the actual read result on the wire.
-   */
-  static void writeReadResult(OutputStream out, Status statusCode)
-      throws IOException {
-    
-    ClientReadStatusProto.newBuilder()
-      .setStatus(statusCode)
-      .build()
-      .writeDelimitedTo(out);
-
-    out.flush();
-  }
-  
-  /**
-   * File name to print when accessing a block directly (from servlets)
-   * @param s Address of the block location
-   * @param poolId Block pool ID of the block
-   * @param blockId Block ID of the block
-   * @return string that has a file name for debug purposes
-   */
-  public static String getFileName(final InetSocketAddress s,
-      final String poolId, final long blockId) {
-    return s.toString() + ":" + poolId + ":" + blockId;
-  }
-
-  @Override
-  public int readAll(byte[] buf, int offset, int len) throws IOException {
-    return BlockReaderUtil.readAll(this, buf, offset, len);
-  }
-
-  @Override
-  public void readFully(byte[] buf, int off, int len) throws IOException {
-    BlockReaderUtil.readFully(this, buf, off, len);
-  }
-  
-  /**
-   * Create a new BlockReader specifically to satisfy a read.
-   * This method also sends the OP_READ_BLOCK request.
-   *
-   * @param file  File location
-   * @param block  The block object
-   * @param blockToken  The block token for security
-   * @param startOffset  The read offset, relative to block head
-   * @param len  The number of bytes to read
-   * @param verifyChecksum  Whether to verify checksum
-   * @param clientName  Client name
-   * @param peer  The Peer to use
-   * @param datanodeID  The DatanodeID this peer is connected to
-   * @return New BlockReader instance, or null on error.
-   */
-  public static BlockReader newBlockReader(String file,
-                                     ExtendedBlock block,
-                                     Token<BlockTokenIdentifier> blockToken,
-                                     long startOffset, long len,
-                                     boolean verifyChecksum,
-                                     String clientName,
-                                     Peer peer, DatanodeID datanodeID,
-                                     PeerCache peerCache,
-                                     CachingStrategy cachingStrategy) throws IOException {
-    // in and out will be closed when sock is closed (by the caller)
-    final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-          peer.getOutputStream()));
-    new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
-        verifyChecksum, cachingStrategy);
-
-    //
-    // Get bytes in block
-    //
-    DataInputStream in = new DataInputStream(peer.getInputStream());
-
-    BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
-        PBHelperClient.vintPrefixed(in));
-    checkSuccess(status, peer, block, file);
-    ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
-    DataChecksum checksum = DataTransferProtoUtil.fromProto(
-        checksumInfo.getChecksum());
-    //Warning when we get CHECKSUM_NULL?
-
-    // Read the first chunk offset.
-    long firstChunkOffset = checksumInfo.getChunkOffset();
-
-    if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
-        firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
-      throw new IOException("BlockReader: error in first chunk offset (" +
-                            firstChunkOffset + ") startOffset is " +
-                            startOffset + " for file " + file);
-    }
-
-    return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
-        checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
-        datanodeID, peerCache);
-  }
-
-  static void checkSuccess(
-      BlockOpResponseProto status, Peer peer,
-      ExtendedBlock block, String file)
-      throws IOException {
-    String logInfo = "for OP_READ_BLOCK"
-      + ", self=" + peer.getLocalAddressString()
-      + ", remote=" + peer.getRemoteAddressString()
-      + ", for file " + file
-      + ", for pool " + block.getBlockPoolId()
-      + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
-    DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
-  }
-  
-  @Override
-  public int available() throws IOException {
-    // An optimistic estimate of how much data is available
-    // to us without doing network I/O.
-    return DFSClient.TCP_WINDOW_SIZE;
-  }
-  
-  @Override
-  public boolean isLocal() {
-    return isLocal;
-  }
-  
-  @Override
-  public boolean isShortCircuit() {
-    return false;
-  }
-
-  @Override
-  public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
deleted file mode 100644
index c9966a7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
+++ /dev/null
@@ -1,214 +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.hdfs.protocol.datatransfer;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto;
-import org.apache.hadoop.hdfs.util.ByteBufferOutputStream;
-
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Shorts;
-import com.google.common.primitives.Ints;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Header data for each packet that goes through the read/write pipelines.
- * Includes all of the information about the packet, excluding checksums and
- * actual data.
- * 
- * This data includes:
- *  - the offset in bytes into the HDFS block of the data in this packet
- *  - the sequence number of this packet in the pipeline
- *  - whether or not this is the last packet in the pipeline
- *  - the length of the data in this packet
- *  - whether or not this packet should be synced by the DNs.
- *  
- * When serialized, this header is written out as a protocol buffer, preceded
- * by a 4-byte integer representing the full packet length, and a 2-byte short
- * representing the header length.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class PacketHeader {
-  private static final int MAX_PROTO_SIZE = 
-    PacketHeaderProto.newBuilder()
-      .setOffsetInBlock(0)
-      .setSeqno(0)
-      .setLastPacketInBlock(false)
-      .setDataLen(0)
-      .setSyncBlock(false)
-      .build().getSerializedSize();
-  public static final int PKT_LENGTHS_LEN =
-      Ints.BYTES + Shorts.BYTES;
-  public static final int PKT_MAX_HEADER_LEN =
-      PKT_LENGTHS_LEN + MAX_PROTO_SIZE;
-
-  private int packetLen;
-  private PacketHeaderProto proto;
-
-  public PacketHeader() {
-  }
-
-  public PacketHeader(int packetLen, long offsetInBlock, long seqno,
-                      boolean lastPacketInBlock, int dataLen, boolean syncBlock) {
-    this.packetLen = packetLen;
-    Preconditions.checkArgument(packetLen >= Ints.BYTES,
-        "packet len %s should always be at least 4 bytes",
-        packetLen);
-    
-    PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
-      .setOffsetInBlock(offsetInBlock)
-      .setSeqno(seqno)
-      .setLastPacketInBlock(lastPacketInBlock)
-      .setDataLen(dataLen);
-      
-    if (syncBlock) {
-      // Only set syncBlock if it is specified.
-      // This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
-      // because it changes the length of the packet header, and BlockReceiver
-      // in that version did not support variable-length headers.
-      builder.setSyncBlock(syncBlock);
-    }
-      
-    proto = builder.build();
-  }
-
-  public int getDataLen() {
-    return proto.getDataLen();
-  }
-
-  public boolean isLastPacketInBlock() {
-    return proto.getLastPacketInBlock();
-  }
-
-  public long getSeqno() {
-    return proto.getSeqno();
-  }
-
-  public long getOffsetInBlock() {
-    return proto.getOffsetInBlock();
-  }
-
-  public int getPacketLen() {
-    return packetLen;
-  }
-
-  public boolean getSyncBlock() {
-    return proto.getSyncBlock();
-  }
-
-  @Override
-  public String toString() {
-    return "PacketHeader with packetLen=" + packetLen +
-      " header data: " + 
-      proto.toString();
-  }
-  
-  public void setFieldsFromData(
-      int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
-    this.packetLen = packetLen;
-    proto = PacketHeaderProto.parseFrom(headerData);
-  }
-  
-  public void readFields(ByteBuffer buf) throws IOException {
-    packetLen = buf.getInt();
-    short protoLen = buf.getShort();
-    byte[] data = new byte[protoLen];
-    buf.get(data);
-    proto = PacketHeaderProto.parseFrom(data);
-  }
-  
-  public void readFields(DataInputStream in) throws IOException {
-    this.packetLen = in.readInt();
-    short protoLen = in.readShort();
-    byte[] data = new byte[protoLen];
-    in.readFully(data);
-    proto = PacketHeaderProto.parseFrom(data);
-  }
-
-  /**
-   * @return the number of bytes necessary to write out this header,
-   * including the length-prefixing of the payload and header
-   */
-  public int getSerializedSize() {
-    return PKT_LENGTHS_LEN + proto.getSerializedSize();
-  }
-
-  /**
-   * Write the header into the buffer.
-   * This requires that PKT_HEADER_LEN bytes are available.
-   */
-  public void putInBuffer(final ByteBuffer buf) {
-    assert proto.getSerializedSize() <= MAX_PROTO_SIZE
-      : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
-    try {
-      buf.putInt(packetLen);
-      buf.putShort((short) proto.getSerializedSize());
-      proto.writeTo(new ByteBufferOutputStream(buf));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-  
-  public void write(DataOutputStream out) throws IOException {
-    assert proto.getSerializedSize() <= MAX_PROTO_SIZE
-    : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
-    out.writeInt(packetLen);
-    out.writeShort(proto.getSerializedSize());
-    proto.writeTo(out);
-  }
-  
-  public byte[] getBytes() {
-    ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
-    putInBuffer(buf);
-    return buf.array();
-  }
-
-  /**
-   * Perform a sanity check on the packet, returning true if it is sane.
-   * @param lastSeqNo the previous sequence number received - we expect the current
-   * sequence number to be larger by 1.
-   */
-  public boolean sanityCheck(long lastSeqNo) {
-    // We should only have a non-positive data length for the last packet
-    if (proto.getDataLen() <= 0 && !proto.getLastPacketInBlock()) return false;
-    // The last packet should not contain data
-    if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
-    // Seqnos should always increase by 1 with each packet received
-    if (proto.getSeqno() != lastSeqNo + 1) return false;
-    return true;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (!(o instanceof PacketHeader)) return false;
-    PacketHeader other = (PacketHeader)o;
-    return this.proto.equals(other.proto);
-  }
-
-  @Override
-  public int hashCode() {
-    return (int)proto.getSeqno();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
deleted file mode 100644
index 3045a13..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
+++ /dev/null
@@ -1,310 +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.hdfs.protocol.datatransfer;
-
-import java.io.Closeable;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.util.DirectBufferPool;
-import org.apache.hadoop.io.IOUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Ints;
-
-/**
- * Class to handle reading packets one-at-a-time from the wire.
- * These packets are used both for reading and writing data to/from
- * DataNodes.
- */
-@InterfaceAudience.Private
-public class PacketReceiver implements Closeable {
-
-  /**
-   * The max size of any single packet. This prevents OOMEs when
-   * invalid data is sent.
-   */
-  private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
-
-  static final Log LOG = LogFactory.getLog(PacketReceiver.class);
-  
-  private static final DirectBufferPool bufferPool = new DirectBufferPool();
-  private final boolean useDirectBuffers;
-
-  /**
-   * The entirety of the most recently read packet.
-   * The first PKT_LENGTHS_LEN bytes of this buffer are the
-   * length prefixes.
-   */
-  private ByteBuffer curPacketBuf = null;
-  
-  /**
-   * A slice of {@link #curPacketBuf} which contains just the checksums.
-   */
-  private ByteBuffer curChecksumSlice = null;
-  
-  /**
-   * A slice of {@link #curPacketBuf} which contains just the data.
-   */
-  private ByteBuffer curDataSlice = null;
-
-  /**
-   * The packet header of the most recently read packet.
-   */
-  private PacketHeader curHeader;
-  
-  public PacketReceiver(boolean useDirectBuffers) {
-    this.useDirectBuffers = useDirectBuffers;
-    reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
-  }
-
-  public PacketHeader getHeader() {
-    return curHeader;
-  }
-
-  public ByteBuffer getDataSlice() {
-    return curDataSlice;
-  }
-  
-  public ByteBuffer getChecksumSlice() {
-    return curChecksumSlice;
-  }
-
-  /**
-   * Reads all of the data for the next packet into the appropriate buffers.
-   * 
-   * The data slice and checksum slice members will be set to point to the
-   * user data and corresponding checksums. The header will be parsed and
-   * set.
-   */
-  public void receiveNextPacket(ReadableByteChannel in) throws IOException {
-    doRead(in, null);
-  }
-
-  /**
-   * @see #receiveNextPacket(ReadableByteChannel)
-   */
-  public void receiveNextPacket(InputStream in) throws IOException {
-    doRead(null, in);
-  }
-
-  private void doRead(ReadableByteChannel ch, InputStream in)
-      throws IOException {
-    // Each packet looks like:
-    //   PLEN    HLEN      HEADER     CHECKSUMS  DATA
-    //   32-bit  16-bit   <protobuf>  <variable length>
-    //
-    // PLEN:      Payload length
-    //            = length(PLEN) + length(CHECKSUMS) + length(DATA)
-    //            This length includes its own encoded length in
-    //            the sum for historical reasons.
-    //
-    // HLEN:      Header length
-    //            = length(HEADER)
-    //
-    // HEADER:    the actual packet header fields, encoded in protobuf
-    // CHECKSUMS: the crcs for the data chunk. May be missing if
-    //            checksums were not requested
-    // DATA       the actual block data
-    Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock());
-
-    curPacketBuf.clear();
-    curPacketBuf.limit(PacketHeader.PKT_LENGTHS_LEN);
-    doReadFully(ch, in, curPacketBuf);
-    curPacketBuf.flip();
-    int payloadLen = curPacketBuf.getInt();
-    
-    if (payloadLen < Ints.BYTES) {
-      // The "payload length" includes its own length. Therefore it
-      // should never be less than 4 bytes
-      throw new IOException("Invalid payload length " +
-          payloadLen);
-    }
-    int dataPlusChecksumLen = payloadLen - Ints.BYTES;
-    int headerLen = curPacketBuf.getShort();
-    if (headerLen < 0) {
-      throw new IOException("Invalid header length " + headerLen);
-    }
-    
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
-          " headerLen = " + headerLen);
-    }
-    
-    // Sanity check the buffer size so we don't allocate too much memory
-    // and OOME.
-    int totalLen = payloadLen + headerLen;
-    if (totalLen < 0 || totalLen > MAX_PACKET_SIZE) {
-      throw new IOException("Incorrect value for packet payload size: " +
-                            payloadLen);
-    }
-
-    // Make sure we have space for the whole packet, and
-    // read it.
-    reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN +
-        dataPlusChecksumLen + headerLen);
-    curPacketBuf.clear();
-    curPacketBuf.position(PacketHeader.PKT_LENGTHS_LEN);
-    curPacketBuf.limit(PacketHeader.PKT_LENGTHS_LEN +
-        dataPlusChecksumLen + headerLen);
-    doReadFully(ch, in, curPacketBuf);
-    curPacketBuf.flip();
-    curPacketBuf.position(PacketHeader.PKT_LENGTHS_LEN);
-
-    // Extract the header from the front of the buffer (after the length prefixes)
-    byte[] headerBuf = new byte[headerLen];
-    curPacketBuf.get(headerBuf);
-    if (curHeader == null) {
-      curHeader = new PacketHeader();
-    }
-    curHeader.setFieldsFromData(payloadLen, headerBuf);
-    
-    // Compute the sub-slices of the packet
-    int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
-    if (checksumLen < 0) {
-      throw new IOException("Invalid packet: data length in packet header " + 
-          "exceeds data length received. dataPlusChecksumLen=" +
-          dataPlusChecksumLen + " header: " + curHeader); 
-    }
-    
-    reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
-  }
-  
-  /**
-   * Rewrite the last-read packet on the wire to the given output stream.
-   */
-  public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException {
-    Preconditions.checkState(!useDirectBuffers,
-        "Currently only supported for non-direct buffers");
-    mirrorOut.write(curPacketBuf.array(),
-        curPacketBuf.arrayOffset(),
-        curPacketBuf.remaining());
-  }
-
-  
-  private static void doReadFully(ReadableByteChannel ch, InputStream in,
-      ByteBuffer buf) throws IOException {
-    if (ch != null) {
-      readChannelFully(ch, buf);
-    } else {
-      Preconditions.checkState(!buf.isDirect(),
-          "Must not use direct buffers with InputStream API");
-      IOUtils.readFully(in, buf.array(),
-          buf.arrayOffset() + buf.position(),
-          buf.remaining());
-      buf.position(buf.position() + buf.remaining());
-    }
-  }
-
-  private void reslicePacket(
-      int headerLen, int checksumsLen, int dataLen) {
-    // Packet structure (refer to doRead() for details):
-    //   PLEN    HLEN      HEADER     CHECKSUMS  DATA
-    //   32-bit  16-bit   <protobuf>  <variable length>
-    //   |--- lenThroughHeader ----|
-    //   |----------- lenThroughChecksums   ----|
-    //   |------------------- lenThroughData    ------| 
-    int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
-    int lenThroughChecksums = lenThroughHeader + checksumsLen;
-    int lenThroughData = lenThroughChecksums + dataLen;
-
-    assert dataLen >= 0 : "invalid datalen: " + dataLen;
-    assert curPacketBuf.position() == lenThroughHeader;
-    assert curPacketBuf.limit() == lenThroughData :
-      "headerLen= " + headerLen + " clen=" + checksumsLen + " dlen=" + dataLen +
-      " rem=" + curPacketBuf.remaining();
-
-    // Slice the checksums.
-    curPacketBuf.position(lenThroughHeader);
-    curPacketBuf.limit(lenThroughChecksums);
-    curChecksumSlice = curPacketBuf.slice();
-
-    // Slice the data.
-    curPacketBuf.position(lenThroughChecksums);
-    curPacketBuf.limit(lenThroughData);
-    curDataSlice = curPacketBuf.slice();
-    
-    // Reset buffer to point to the entirety of the packet (including
-    // length prefixes)
-    curPacketBuf.position(0);
-    curPacketBuf.limit(lenThroughData);
-  }
-
-  
-  private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
-      throws IOException {
-    while (buf.remaining() > 0) {
-      int n = ch.read(buf);
-      if (n < 0) {
-        throw new IOException("Premature EOF reading from " + ch);
-      }
-    }
-  }
-  
-  private void reallocPacketBuf(int atLeastCapacity) {
-    // Realloc the buffer if this packet is longer than the previous
-    // one.
-    if (curPacketBuf == null ||
-        curPacketBuf.capacity() < atLeastCapacity) {
-      ByteBuffer newBuf;
-      if (useDirectBuffers) {
-        newBuf = bufferPool.getBuffer(atLeastCapacity);
-      } else {
-        newBuf = ByteBuffer.allocate(atLeastCapacity);
-      }
-      // If reallocing an existing buffer, copy the old packet length
-      // prefixes over
-      if (curPacketBuf != null) {
-        curPacketBuf.flip();
-        newBuf.put(curPacketBuf);
-      }
-      
-      returnPacketBufToPool();
-      curPacketBuf = newBuf;
-    }
-  }
-  
-  private void returnPacketBufToPool() {
-    if (curPacketBuf != null && curPacketBuf.isDirect()) {
-      bufferPool.returnBuffer(curPacketBuf);
-      curPacketBuf = null;
-    }
-  }
-
-  @Override // Closeable
-  public void close() {
-    returnPacketBufToPool();
-  }
-  
-  @Override
-  protected void finalize() throws Throwable {
-    try {
-      // just in case it didn't get closed, we
-      // may as well still try to return the buffer
-      returnPacketBufToPool();
-    } finally {
-      super.finalize();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
deleted file mode 100644
index 31d4dcc..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
+++ /dev/null
@@ -1,49 +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.hdfs.util;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * OutputStream that writes into a {@link ByteBuffer}.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ByteBufferOutputStream extends OutputStream {
-
-  private final ByteBuffer buf;
-
-  public ByteBufferOutputStream(ByteBuffer buf) {
-    this.buf = buf;
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    buf.put((byte)b);
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    buf.put(b, off, len);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java
index 8dd3d6f..5ff343a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java
@@ -24,10 +24,10 @@ import static org.mockito.Mockito.verify;
 
 import java.util.List;
 
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -41,7 +41,7 @@ public class TestClientBlockVerification {
   static LocatedBlock testBlock = null;
 
   static {
-    ((Log4JLogger)RemoteBlockReader2.LOG).getLogger().setLevel(Level.ALL);
+    GenericTestUtils.setLogLevel(RemoteBlockReader2.LOG, Level.ALL);
   }
   @BeforeClass
   public static void setupCluster() throws Exception {


[36/50] [abbrv] hadoop git commit: HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 6f16d83..5b60307 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.protocolPB;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
     .EncryptionZoneProto;
-import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherSuiteProto;
 import static org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
 
 import java.io.IOException;
@@ -2315,7 +2314,7 @@ public class PBHelper {
     return EncryptionZoneProto.newBuilder()
         .setId(zone.getId())
         .setPath(zone.getPath())
-        .setSuite(convert(zone.getSuite()))
+        .setSuite(PBHelperClient.convert(zone.getSuite()))
         .setCryptoProtocolVersion(convert(zone.getVersion()))
         .setKeyName(zone.getKeyName())
         .build();
@@ -2323,7 +2322,7 @@ public class PBHelper {
 
   public static EncryptionZone convert(EncryptionZoneProto proto) {
     return new EncryptionZone(proto.getId(), proto.getPath(),
-        convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
+        PBHelperClient.convert(proto.getSuite()), convert(proto.getCryptoProtocolVersion()),
         proto.getKeyName());
   }
 
@@ -2624,106 +2623,6 @@ public class PBHelper {
     return GetEditsFromTxidResponseProto.newBuilder().setEventsList(
         builder.build()).build();
   }
-  
-  public static CipherOptionProto convert(CipherOption option) {
-    if (option != null) {
-      CipherOptionProto.Builder builder = CipherOptionProto.
-          newBuilder();
-      if (option.getCipherSuite() != null) {
-        builder.setSuite(convert(option.getCipherSuite()));
-      }
-      if (option.getInKey() != null) {
-        builder.setInKey(ByteString.copyFrom(option.getInKey()));
-      }
-      if (option.getInIv() != null) {
-        builder.setInIv(ByteString.copyFrom(option.getInIv()));
-      }
-      if (option.getOutKey() != null) {
-        builder.setOutKey(ByteString.copyFrom(option.getOutKey()));
-      }
-      if (option.getOutIv() != null) {
-        builder.setOutIv(ByteString.copyFrom(option.getOutIv()));
-      }
-      return builder.build();
-    }
-    return null;
-  }
-  
-  public static CipherOption convert(CipherOptionProto proto) {
-    if (proto != null) {
-      CipherSuite suite = null;
-      if (proto.getSuite() != null) {
-        suite = convert(proto.getSuite());
-      }
-      byte[] inKey = null;
-      if (proto.getInKey() != null) {
-        inKey = proto.getInKey().toByteArray();
-      }
-      byte[] inIv = null;
-      if (proto.getInIv() != null) {
-        inIv = proto.getInIv().toByteArray();
-      }
-      byte[] outKey = null;
-      if (proto.getOutKey() != null) {
-        outKey = proto.getOutKey().toByteArray();
-      }
-      byte[] outIv = null;
-      if (proto.getOutIv() != null) {
-        outIv = proto.getOutIv().toByteArray();
-      }
-      return new CipherOption(suite, inKey, inIv, outKey, outIv);
-    }
-    return null;
-  }
-  
-  public static List<CipherOptionProto> convertCipherOptions(
-      List<CipherOption> options) {
-    if (options != null) {
-      List<CipherOptionProto> protos = 
-          Lists.newArrayListWithCapacity(options.size());
-      for (CipherOption option : options) {
-        protos.add(convert(option));
-      }
-      return protos;
-    }
-    return null;
-  }
-  
-  public static List<CipherOption> convertCipherOptionProtos(
-      List<CipherOptionProto> protos) {
-    if (protos != null) {
-      List<CipherOption> options = 
-          Lists.newArrayListWithCapacity(protos.size());
-      for (CipherOptionProto proto : protos) {
-        options.add(convert(proto));
-      }
-      return options;
-    }
-    return null;
-  }
-
-  public static CipherSuiteProto convert(CipherSuite suite) {
-    switch (suite) {
-    case UNKNOWN:
-      return CipherSuiteProto.UNKNOWN;
-    case AES_CTR_NOPADDING:
-      return CipherSuiteProto.AES_CTR_NOPADDING;
-    default:
-      return null;
-    }
-  }
-
-  public static CipherSuite convert(CipherSuiteProto proto) {
-    switch (proto) {
-    case AES_CTR_NOPADDING:
-      return CipherSuite.AES_CTR_NOPADDING;
-    default:
-      // Set to UNKNOWN and stash the unknown enum value
-      CipherSuite suite = CipherSuite.UNKNOWN;
-      suite.setUnknownValue(proto.getNumber());
-      return suite;
-    }
-  }
 
   public static List<CryptoProtocolVersionProto> convert(
       CryptoProtocolVersion[] versions) {
@@ -2776,7 +2675,7 @@ public class PBHelper {
       return null;
     }
     return HdfsProtos.FileEncryptionInfoProto.newBuilder()
-        .setSuite(convert(info.getCipherSuite()))
+        .setSuite(PBHelperClient.convert(info.getCipherSuite()))
         .setCryptoProtocolVersion(convert(info.getCryptoProtocolVersion()))
         .setKey(getByteString(info.getEncryptedDataEncryptionKey()))
         .setIv(getByteString(info.getIV()))
@@ -2803,7 +2702,7 @@ public class PBHelper {
       return null;
     }
     return HdfsProtos.ZoneEncryptionInfoProto.newBuilder()
-        .setSuite(convert(suite))
+        .setSuite(PBHelperClient.convert(suite))
         .setCryptoProtocolVersion(convert(version))
         .setKeyName(keyName)
         .build();
@@ -2814,7 +2713,7 @@ public class PBHelper {
     if (proto == null) {
       return null;
     }
-    CipherSuite suite = convert(proto.getSuite());
+    CipherSuite suite = PBHelperClient.convert(proto.getSuite());
     CryptoProtocolVersion version = convert(proto.getCryptoProtocolVersion());
     byte[] key = proto.getKey().toByteArray();
     byte[] iv = proto.getIv().toByteArray();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index e88fa49..ced085a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.security.AccessControlException;
 
 import java.io.IOException;
@@ -282,10 +283,9 @@ class FSDirXAttrOp {
         final HdfsProtos.ZoneEncryptionInfoProto ezProto =
             HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xattr.getValue());
         fsd.ezManager.addEncryptionZone(inode.getId(),
-                                        PBHelper.convert(ezProto.getSuite()),
-                                        PBHelper.convert(
-                                            ezProto.getCryptoProtocolVersion()),
-                                        ezProto.getKeyName());
+            PBHelperClient.convert(ezProto.getSuite()),
+            PBHelper.convert(ezProto.getCryptoProtocolVersion()),
+            ezProto.getKeyName());
       }
 
       if (!isFile && SECURITY_XATTR_UNREADABLE_BY_SUPERUSER.equals(xaName)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 734d3c0..8c74e48 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -1004,7 +1005,7 @@ public class FSDirectory implements Closeable {
                   HdfsProtos.ZoneEncryptionInfoProto.parseFrom(
                       xattr.getValue());
               ezManager.unprotectedAddEncryptionZone(inode.getId(),
-                  PBHelper.convert(ezProto.getSuite()),
+                  PBHelperClient.convert(ezProto.getSuite()),
                   PBHelper.convert(ezProto.getCryptoProtocolVersion()),
                   ezProto.getKeyName());
             } catch (InvalidProtocolBufferException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 7ebe859..b5dbc46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -45,11 +45,11 @@ import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -884,7 +884,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                 try {
                   s.connect(addr, HdfsConstants.READ_TIMEOUT);
                   s.setSoTimeout(HdfsConstants.READ_TIMEOUT);
-                  peer = TcpPeerServer.peerFromSocketAndKey(
+                  peer = DFSUtilClient.peerFromSocketAndKey(
                         dfs.getSaslDataTransferClient(), s, NamenodeFsck.this,
                         blockToken, datanodeId);
                 } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
index 6d8bec3..7b0ef02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java
@@ -32,7 +32,6 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -204,7 +203,7 @@ public class BlockReaderTestUtil {
           try {
             sock.connect(addr, HdfsConstants.READ_TIMEOUT);
             sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
-            peer = TcpPeerServer.peerFromSocket(sock);
+            peer = DFSUtilClient.peerFromSocket(sock);
           } finally {
             if (peer == null) {
               IOUtils.closeQuietly(sock);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index 24e0965..40414c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -21,13 +21,13 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
index bf19c40..9c02e9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
@@ -77,7 +77,7 @@ public class TestEncryptedTransfer {
     conf.setBoolean(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     if (resolverClazz != null){
-      conf.set(DFSConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS, resolverClazz);
+      conf.set(HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS, resolverClazz);
     }
   }
   
@@ -209,7 +209,7 @@ public class TestEncryptedTransfer {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      conf.set(DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY,
+      conf.set(HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY,
           "AES/CTR/NoPadding");
       cluster = new MiniDFSCluster.Builder(conf).build();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
index 403b21e..9e660b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferTestCase.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hdfs.protocol.datatransfer.sasl;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
index 16ddc75..0cbf266 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer.sasl;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.IGNORE_SECURE_PORTS_FOR_TESTING_KEY;
 import static org.junit.Assert.assertArrayEquals;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
index 1da92a1..b40e52a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.qjournal;
 import static org.junit.Assert.*;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
index 234eb22..79a15e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
@@ -41,12 +41,12 @@ import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -171,7 +171,7 @@ public class TestBlockTokenWithDFS {
               try {
                 sock.connect(addr, HdfsConstants.READ_TIMEOUT);
                 sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
-                peer = TcpPeerServer.peerFromSocket(sock);
+                peer = DFSUtilClient.peerFromSocket(sock);
               } finally {
                 if (peer == null) {
                   IOUtils.closeSocket(sock);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 4e37161..cb50edc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -43,12 +43,12 @@ import org.apache.hadoop.hdfs.BlockReaderFactory;
 import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.RemotePeerFactory;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -525,7 +525,7 @@ public class TestDataNodeVolumeFailure {
           try {
             sock.connect(addr, HdfsConstants.READ_TIMEOUT);
             sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
-            peer = TcpPeerServer.peerFromSocket(sock);
+            peer = DFSUtilClient.peerFromSocket(sock);
           } finally {
             if (peer == null) {
               IOUtils.closeSocket(sock);


[08/50] [abbrv] hadoop git commit: HDFS-8946. Improve choosing datanode storage for block placement. (yliu)

Posted by wa...@apache.org.
HDFS-8946. Improve choosing datanode storage for block placement. (yliu)


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

Branch: refs/heads/YARN-1197
Commit: 8fa41d9dd4b923bf4141f019414a1a8b079124c6
Parents: 4eaa7fd
Author: yliu <yl...@apache.org>
Authored: Tue Sep 1 08:52:50 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue Sep 1 08:52:50 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../BlockPlacementPolicyDefault.java            | 147 ++++++-------------
 .../blockmanagement/DatanodeDescriptor.java     |  36 +++--
 .../blockmanagement/TestReplicationPolicy.java  |  26 +++-
 4 files changed, 93 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fa41d9d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ef8fac5..6584c84 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -870,6 +870,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8990. Move RemoteBlockReader to hdfs-client module.
     (Mingliang via wheat9)
 
+    HDFS-8946. Improve choosing datanode storage for block placement. (yliu)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fa41d9d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 6d7a765..f761150 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -26,12 +26,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
@@ -458,19 +455,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
             .entrySet().iterator(); iter.hasNext(); ) {
           Map.Entry<StorageType, Integer> entry = iter.next();
-          for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
-              localDatanode.getStorageInfos())) {
-            StorageType type = entry.getKey();
-            if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
-                results, type) >= 0) {
-              int num = entry.getValue();
-              if (num == 1) {
-                iter.remove();
-              } else {
-                entry.setValue(num - 1);
-              }
-              return localStorage;
+          DatanodeStorageInfo localStorage = chooseStorage4Block(
+              localDatanode, blocksize, results, entry.getKey());
+          if (localStorage != null) {
+            // add node and related nodes to excludedNode
+            addToExcludedNodes(localDatanode, excludedNodes);
+            int num = entry.getValue();
+            if (num == 1) {
+              iter.remove();
+            } else {
+              entry.setValue(num - 1);
             }
+            return localStorage;
           }
         }
       } 
@@ -651,7 +647,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                             boolean avoidStaleNodes,
                             EnumMap<StorageType, Integer> storageTypes)
                             throws NotEnoughReplicasException {
-      
+
     int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
         scope, excludedNodes);
     StringBuilder builder = null;
@@ -669,49 +665,39 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
           builder.append("\nNode ").append(NodeBase.getPath(chosenNode)).append(" [");
         }
         numOfAvailableNodes--;
-        if (!isGoodDatanode(chosenNode, maxNodesPerRack, considerLoad,
+        DatanodeStorageInfo storage = null;
+        if (isGoodDatanode(chosenNode, maxNodesPerRack, considerLoad,
             results, avoidStaleNodes)) {
-          if (LOG.isDebugEnabled()) {
-            builder.append("\n]");
-          }
-          badTarget = true;
-          continue;
-        }
-
-        final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
-            chosenNode.getStorageInfos());
-        int i = 0;
-        boolean search = true;
-        for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
-            .entrySet().iterator(); search && iter.hasNext(); ) {
-          Map.Entry<StorageType, Integer> entry = iter.next();
-          for (i = 0; i < storages.length; i++) {
-            StorageType type = entry.getKey();
-            final int newExcludedNodes = addIfIsGoodTarget(storages[i],
-                excludedNodes, blocksize, results, type);
-            if (newExcludedNodes >= 0) {
+          for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
+              .entrySet().iterator(); iter.hasNext(); ) {
+            Map.Entry<StorageType, Integer> entry = iter.next();
+            storage = chooseStorage4Block(
+                chosenNode, blocksize, results, entry.getKey());
+            if (storage != null) {
               numOfReplicas--;
               if (firstChosen == null) {
-                firstChosen = storages[i];
+                firstChosen = storage;
               }
-              numOfAvailableNodes -= newExcludedNodes;
+              // add node and related nodes to excludedNode
+              numOfAvailableNodes -=
+                  addToExcludedNodes(chosenNode, excludedNodes);
               int num = entry.getValue();
               if (num == 1) {
                 iter.remove();
               } else {
                 entry.setValue(num - 1);
               }
-              search = false;
               break;
             }
           }
         }
+
         if (LOG.isDebugEnabled()) {
           builder.append("\n]");
         }
 
         // If no candidate storage was found on this DN then set badTarget.
-        badTarget = (i == storages.length);
+        badTarget = (storage == null);
       }
     }
       
@@ -740,32 +726,27 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   /**
-   * If the given storage is a good target, add it to the result list and
-   * update the set of excluded nodes.
-   * @return -1 if the given is not a good target;
-   *         otherwise, return the number of nodes added to excludedNodes set.
+   * Choose a good storage of given storage type from datanode, and add it to
+   * the result list.
+   *
+   * @param dnd datanode descriptor
+   * @param blockSize requested block size
+   * @param results the result storages
+   * @param storageType requested storage type
+   * @return the chosen datanode storage
    */
-  int addIfIsGoodTarget(DatanodeStorageInfo storage,
-      Set<Node> excludedNodes,
+  DatanodeStorageInfo chooseStorage4Block(DatanodeDescriptor dnd,
       long blockSize,
       List<DatanodeStorageInfo> results,
       StorageType storageType) {
-    if (isGoodTarget(storage, blockSize, results, storageType)) {
+    DatanodeStorageInfo storage =
+        dnd.chooseStorage4Block(storageType, blockSize);
+    if (storage != null) {
       results.add(storage);
-      // add node and related nodes to excludedNode
-      return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
-    } else { 
-      return -1;
-    }
-  }
-
-  private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
-    if (LOG.isDebugEnabled()) {
-      // build the error message for later use.
-      debugLoggingBuilder.get()
-          .append("\n  Storage ").append(storage)
-          .append(" is not chosen since ").append(reason).append(".");
+    } else {
+      logNodeIsNotChosen(dnd, "no good storage to place the block ");
     }
+    return storage;
   }
 
   private static void logNodeIsNotChosen(DatanodeDescriptor node,
@@ -837,52 +818,6 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   /**
-   * Determine if a storage is a good target.
-   *
-   * @param storage The target storage
-   * @param blockSize Size of block
-   * @param results A list containing currently chosen nodes. Used to check if
-   *                too many nodes has been chosen in the target rack.
-   * @return Return true if <i>node</i> has enough space.
-   */
-  private boolean isGoodTarget(DatanodeStorageInfo storage,
-                               long blockSize,
-                               List<DatanodeStorageInfo> results,
-                               StorageType requiredStorageType) {
-    if (storage.getStorageType() != requiredStorageType) {
-      logNodeIsNotChosen(storage, "storage types do not match,"
-          + " where the required storage type is " + requiredStorageType);
-      return false;
-    }
-    if (storage.getState() == State.READ_ONLY_SHARED) {
-      logNodeIsNotChosen(storage, "storage is read-only");
-      return false;
-    }
-
-    if (storage.getState() == State.FAILED) {
-      logNodeIsNotChosen(storage, "storage has failed");
-      return false;
-    }
-
-    DatanodeDescriptor node = storage.getDatanodeDescriptor();
-
-    final long requiredSize = blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE;
-    final long scheduledSize = blockSize * node.getBlocksScheduled(storage.getStorageType());
-    final long remaining = node.getRemaining(storage.getStorageType(),
-        requiredSize);
-    if (requiredSize > remaining - scheduledSize) {
-      logNodeIsNotChosen(storage, "the node does not have enough "
-          + storage.getStorageType() + " space"
-          + " (required=" + requiredSize
-          + ", scheduled=" + scheduledSize
-          + ", remaining=" + remaining + ")");
-      return false;
-    }
-
-    return true;
-  }
-
-  /**
    * Return a pipeline of nodes.
    * The pipeline is formed finding a shortest path that 
    * starts from the writer and traverses all <i>nodes</i>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fa41d9d/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 7e3c59b..0b398c5 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
@@ -31,14 +31,15 @@ import java.util.Queue;
 import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import com.google.common.collect.ImmutableList;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -663,26 +664,39 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Return the sum of remaining spaces of the specified type. If the remaining
-   * space of a storage is less than minSize, it won't be counted toward the
-   * sum.
+   * Find whether the datanode contains good storage of given type to
+   * place block of size <code>blockSize</code>.
    *
-   * @param t The storage type. If null, the type is ignored.
-   * @param minSize The minimum free space required.
-   * @return the sum of remaining spaces that are bigger than minSize.
+   * <p>Currently datanode only cares about the storage type, in this
+   * method, the first storage of given type we see is returned.
+   *
+   * @param t requested storage type
+   * @param blockSize requested block size
+   * @return
    */
-  public long getRemaining(StorageType t, long minSize) {
+  public DatanodeStorageInfo chooseStorage4Block(StorageType t,
+      long blockSize) {
+    final long requiredSize =
+        blockSize * HdfsServerConstants.MIN_BLOCKS_FOR_WRITE;
+    final long scheduledSize = blockSize * getBlocksScheduled(t);
     long remaining = 0;
+    DatanodeStorageInfo storage = null;
     for (DatanodeStorageInfo s : getStorageInfos()) {
       if (s.getState() == State.NORMAL &&
-          (t == null || s.getStorageType() == t)) {
+          s.getStorageType() == t) {
+        if (storage == null) {
+          storage = s;
+        }
         long r = s.getRemaining();
-        if (r >= minSize) {
+        if (r >= requiredSize) {
           remaining += r;
         }
       }
     }
-    return remaining;
+    if (requiredSize > remaining - scheduledSize) {
+      return null;
+    }
+    return storage;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8fa41d9d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index cec33fe..27d647c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -181,7 +181,7 @@ public class TestReplicationPolicy {
    * considered.
    */
   @Test
-  public void testChooseNodeWithMultipleStorages() throws Exception {
+  public void testChooseNodeWithMultipleStorages1() throws Exception {
     updateHeartbeatWithUsage(dataNodes[5],
         2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
         (2*HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE)/3, 0L,
@@ -201,6 +201,30 @@ public class TestReplicationPolicy {
   }
 
   /**
+   * Test whether all storages on the datanode are considered while
+   * choosing target to place block.
+   */
+  @Test
+  public void testChooseNodeWithMultipleStorages2() throws Exception {
+    updateHeartbeatWithUsage(dataNodes[5],
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        (2*HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE)/3, 0L,
+        0L, 0L, 0, 0);
+
+    updateHeartbeatForExtraStorage(
+        2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
+        HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L);
+
+    DatanodeStorageInfo[] targets;
+    targets = chooseTarget (1, dataNodes[5],
+        new ArrayList<DatanodeStorageInfo>(), null);
+    assertEquals(1, targets.length);
+    assertEquals(dataNodes[5], targets[0].getDatanodeDescriptor());
+
+    resetHeartbeatForStorages();
+  }
+
+  /**
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * placed on dataNodes[0], the 2nd replica should be placed on 
    * different rack and third should be placed on different node


[47/50] [abbrv] hadoop git commit: YARN-1645. ContainerManager implementation to support container resizing. Contributed by Meng Ding & Wangda Tan

Posted by wa...@apache.org.
YARN-1645. ContainerManager implementation to support container resizing. Contributed by Meng Ding & Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: b05443ef98393d011b7cc5694d0e8875ef832f42
Parents: 016da56
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 21 16:10:40 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:03 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../CMgrDecreaseContainersResourceEvent.java    |  37 ++++
 .../nodemanager/ContainerManagerEventType.java  |   1 +
 .../containermanager/ContainerManagerImpl.java  | 180 ++++++++++++++++--
 .../container/ChangeContainerResourceEvent.java |  36 ++++
 .../container/ContainerEventType.java           |   4 +
 .../nodemanager/DummyContainerManager.java      |   6 +-
 .../TestContainerManagerWithLCE.java            |  22 +++
 .../BaseContainerManagerTest.java               |  43 ++++-
 .../containermanager/TestContainerManager.java  | 190 ++++++++++++++++++-
 10 files changed, 486 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index ed81a02..e17f413 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -196,6 +196,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1449. AM-NM protocol changes to support container resizing.
     (Meng Ding & Wangda Tan via jianhe)
 
+    YARN-1645. ContainerManager implementation to support container resizing.
+    (Meng Ding & Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.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/CMgrDecreaseContainersResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
new file mode 100644
index 0000000..9479d0b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.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.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import java.util.List;
+
+public class CMgrDecreaseContainersResourceEvent extends ContainerManagerEvent {
+
+  private final List<Container> containersToDecrease;
+
+  public CMgrDecreaseContainersResourceEvent(List<Container>
+      containersToDecrease) {
+    super(ContainerManagerEventType.DECREASE_CONTAINERS_RESOURCE);
+    this.containersToDecrease = containersToDecrease;
+  }
+
+  public List<Container> getContainersToDecrease() {
+    return this.containersToDecrease;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.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/ContainerManagerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
index 4278ce0..fcb0252 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
@@ -21,4 +21,5 @@ package org.apache.hadoop.yarn.server.nodemanager;
 public enum ContainerManagerEventType {
   FINISH_APPS,
   FINISH_CONTAINERS,
+  DECREASE_CONTAINERS_RESOURCE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index ce68b4e..e7572b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
@@ -96,6 +97,7 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent;
+import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -113,6 +115,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ChangeContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
@@ -140,6 +143,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerManagerImpl extends CompositeService implements
     ServiceStateChangeListener, ContainerManagementProtocol,
@@ -659,33 +663,45 @@ public class ContainerManagerImpl extends CompositeService implements
 
   /**
    * @param containerTokenIdentifier
-   *          of the container to be started
+   *          of the container whose resource is to be started or increased
    * @throws YarnException
    */
   @Private
   @VisibleForTesting
-  protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
+  protected void authorizeStartAndResourceIncreaseRequest(
+      NMTokenIdentifier nmTokenIdentifier,
+      ContainerTokenIdentifier containerTokenIdentifier,
+      boolean startRequest)
+      throws YarnException {
     if (nmTokenIdentifier == null) {
       throw RPCUtil.getRemoteException(INVALID_NMTOKEN_MSG);
     }
     if (containerTokenIdentifier == null) {
       throw RPCUtil.getRemoteException(INVALID_CONTAINERTOKEN_MSG);
     }
+    /*
+     * Check the following:
+     * 1. The request comes from the same application attempt
+     * 2. The request possess a container token that has not expired
+     * 3. The request possess a container token that is granted by a known RM
+     */
     ContainerId containerId = containerTokenIdentifier.getContainerID();
     String containerIDStr = containerId.toString();
     boolean unauthorized = false;
     StringBuilder messageBuilder =
-        new StringBuilder("Unauthorized request to start container. ");
+        new StringBuilder("Unauthorized request to " + (startRequest ?
+            "start container." : "increase container resource."));
     if (!nmTokenIdentifier.getApplicationAttemptId().getApplicationId().
         equals(containerId.getApplicationAttemptId().getApplicationId())) {
       unauthorized = true;
       messageBuilder.append("\nNMToken for application attempt : ")
         .append(nmTokenIdentifier.getApplicationAttemptId())
-        .append(" was used for starting container with container token")
+        .append(" was used for "
+            + (startRequest ? "starting " : "increasing resource of ")
+            + "container with container token")
         .append(" issued for application attempt : ")
         .append(containerId.getApplicationAttemptId());
-    } else if (!this.context.getContainerTokenSecretManager()
+    } else if (startRequest && !this.context.getContainerTokenSecretManager()
         .isValidStartContainerRequest(containerTokenIdentifier)) {
       // Is the container being relaunched? Or RPC layer let startCall with
       // tokens generated off old-secret through?
@@ -707,6 +723,14 @@ public class ContainerManagerImpl extends CompositeService implements
       LOG.error(msg);
       throw RPCUtil.getRemoteException(msg);
     }
+    if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater
+        .getRMIdentifier()) {
+      // Is the container coming from unknown RM
+      StringBuilder sb = new StringBuilder("\nContainer ");
+      sb.append(containerTokenIdentifier.getContainerID().toString())
+        .append(" rejected as it is allocated by a previous RM");
+      throw new InvalidContainerException(sb.toString());
+    }
   }
 
   /**
@@ -723,7 +747,7 @@ public class ContainerManagerImpl extends CompositeService implements
     }
     UserGroupInformation remoteUgi = getRemoteUgi();
     NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
-    authorizeUser(remoteUgi,nmTokenIdentifier);
+    authorizeUser(remoteUgi, nmTokenIdentifier);
     List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
@@ -813,16 +837,8 @@ public class ContainerManagerImpl extends CompositeService implements
      * belongs to correct Node Manager (part of retrieve password). c) It has
      * correct RMIdentifier. d) It is not expired.
      */
-    authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier);
- 
-    if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater
-        .getRMIdentifier()) {
-        // Is the container coming from unknown RM
-        StringBuilder sb = new StringBuilder("\nContainer ");
-        sb.append(containerTokenIdentifier.getContainerID().toString())
-          .append(" rejected as it is allocated by a previous RM");
-        throw new InvalidContainerException(sb.toString());
-    }
+    authorizeStartAndResourceIncreaseRequest(
+        nmTokenIdentifier, containerTokenIdentifier, true);
     // update NMToken
     updateNMTokenIdentifier(nmTokenIdentifier);
 
@@ -928,9 +944,118 @@ public class ContainerManagerImpl extends CompositeService implements
   @Override
   public IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest requests)
-      throws YarnException, IOException {
-    // To be implemented in YARN-1645
-    return null;
+          throws YarnException, IOException {
+    if (blockNewContainerRequests.get()) {
+      throw new NMNotYetReadyException(
+          "Rejecting container resource increase as NodeManager has not"
+              + " yet connected with ResourceManager");
+    }
+    UserGroupInformation remoteUgi = getRemoteUgi();
+    NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
+    authorizeUser(remoteUgi, nmTokenIdentifier);
+    List<ContainerId> successfullyIncreasedContainers
+        = new ArrayList<ContainerId>();
+    Map<ContainerId, SerializedException> failedContainers =
+        new HashMap<ContainerId, SerializedException>();
+    // Process container resource increase requests
+    for (org.apache.hadoop.yarn.api.records.Token token :
+        requests.getContainersToIncrease()) {
+      ContainerId containerId = null;
+      try {
+        if (token.getIdentifier() == null) {
+          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+        }
+        ContainerTokenIdentifier containerTokenIdentifier =
+            BuilderUtils.newContainerTokenIdentifier(token);
+        verifyAndGetContainerTokenIdentifier(token,
+            containerTokenIdentifier);
+        authorizeStartAndResourceIncreaseRequest(
+            nmTokenIdentifier, containerTokenIdentifier, false);
+        containerId = containerTokenIdentifier.getContainerID();
+        // Reuse the startContainer logic to update NMToken,
+        // as container resource increase request will have come with
+        // an updated NMToken.
+        updateNMTokenIdentifier(nmTokenIdentifier);
+        Resource resource = containerTokenIdentifier.getResource();
+        changeContainerResourceInternal(containerId, resource, true);
+        successfullyIncreasedContainers.add(containerId);
+      } catch (YarnException | InvalidToken e) {
+        failedContainers.put(containerId, SerializedException.newInstance(e));
+      } catch (IOException e) {
+        throw RPCUtil.getRemoteException(e);
+      }
+    }
+    return IncreaseContainersResourceResponse.newInstance(
+        successfullyIncreasedContainers, failedContainers);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void changeContainerResourceInternal(
+      ContainerId containerId, Resource targetResource, boolean increase)
+          throws YarnException, IOException {
+    Container container = context.getContainers().get(containerId);
+    // Check container existence
+    if (container == null) {
+      if (nodeStatusUpdater.isContainerRecentlyStopped(containerId)) {
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " was recently stopped on node manager.");
+      } else {
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " is not handled by this NodeManager");
+      }
+    }
+    // Check container state
+    org.apache.hadoop.yarn.server.nodemanager.
+        containermanager.container.ContainerState currentState =
+        container.getContainerState();
+    if (currentState != org.apache.hadoop.yarn.server.
+        nodemanager.containermanager.container.ContainerState.RUNNING) {
+      throw RPCUtil.getRemoteException("Container " + containerId.toString()
+          + " is in " + currentState.name() + " state."
+          + " Resource can only be changed when a container is in"
+          + " RUNNING state");
+    }
+    // Check validity of the target resource.
+    Resource currentResource = container.getResource();
+    if (currentResource.equals(targetResource)) {
+      LOG.warn("Unable to change resource for container "
+          + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is the same as the current resource");
+      return;
+    }
+    if (increase && !Resources.fitsIn(currentResource, targetResource)) {
+      throw RPCUtil.getRemoteException("Unable to increase resource for "
+          + "container " + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is smaller than the current resource "
+          + currentResource.toString());
+    }
+    if (!increase &&
+        (!Resources.fitsIn(Resources.none(), targetResource)
+            || !Resources.fitsIn(targetResource, currentResource))) {
+      throw RPCUtil.getRemoteException("Unable to decrease resource for "
+          + "container " + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is not smaller than the current resource "
+          + currentResource.toString());
+    }
+    this.readLock.lock();
+    try {
+      if (!serviceStopped) {
+        dispatcher.getEventHandler().handle(new ChangeContainerResourceEvent(
+            containerId, targetResource));
+      } else {
+        throw new YarnException(
+            "Unable to change container resource as the NodeManager is "
+                + "in the process of shutting down");
+      }
+    } finally {
+      this.readLock.unlock();
+    }
   }
 
   @Private
@@ -1171,6 +1296,21 @@ public class ContainerManagerImpl extends CompositeService implements
                   "Container Killed by ResourceManager"));
       }
       break;
+    case DECREASE_CONTAINERS_RESOURCE:
+      CMgrDecreaseContainersResourceEvent containersDecreasedEvent =
+          (CMgrDecreaseContainersResourceEvent) event;
+      for (org.apache.hadoop.yarn.api.records.Container container
+          : containersDecreasedEvent.getContainersToDecrease()) {
+        try {
+          changeContainerResourceInternal(container.getId(),
+              container.getResource(), false);
+        } catch (YarnException e) {
+          LOG.error("Unable to decrease container resource", e);
+        } catch (IOException e) {
+          LOG.error("Unable to update container resource in store", e);
+        }
+      }
+      break;
     default:
         throw new YarnRuntimeException(
             "Got an unknown ContainerManagerEvent type: " + event.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.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/container/ChangeContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
new file mode 100644
index 0000000..3944a3d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.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.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class ChangeContainerResourceEvent extends ContainerEvent {
+
+  private Resource resource;
+
+  public ChangeContainerResourceEvent(ContainerId c, Resource resource) {
+    super(c, ContainerEventType.CHANGE_CONTAINER_RESOURCE);
+    this.resource = resource;
+  }
+
+  public Resource getResource() {
+    return this.resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.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/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index 5622f8c..dc712bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -25,6 +25,10 @@ public enum ContainerEventType {
   KILL_CONTAINER,
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
+  CHANGE_CONTAINER_RESOURCE,
+
+  // Producer: ContainerMonitor
+  CONTAINER_RESOURCE_CHANGED,
 
   // DownloadManager
   CONTAINER_INITED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.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/DummyContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
index 349340b..3ff04d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
@@ -191,8 +191,10 @@ public class DummyContainerManager extends ContainerManagerImpl {
   }
   
   @Override
-  protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
+  protected void authorizeStartAndResourceIncreaseRequest(
+      NMTokenIdentifier nmTokenIdentifier,
+      ContainerTokenIdentifier containerTokenIdentifier,
+      boolean startRequest) throws YarnException {
     // do nothing
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.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/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index a47e7f7..9a05278 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -189,6 +189,28 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     super.testStartContainerFailureWithUnknownAuxService();
   }
 
+  @Override
+  public void testIncreaseContainerResourceWithInvalidRequests() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testIncreaseContainerResourceWithInvalidRequests");
+    super.testIncreaseContainerResourceWithInvalidRequests();
+  }
+
+  @Override
+  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testIncreaseContainerResourceWithInvalidResource");
+    super.testIncreaseContainerResourceWithInvalidResource();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.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/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index 2810662..3938342 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -209,12 +209,13 @@ public abstract class BaseContainerManagerTest {
         // do nothing
       }
       @Override
-        protected void authorizeStartRequest(
-            NMTokenIdentifier nmTokenIdentifier,
-            ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
-          // do nothing
-        }
-      
+      protected void authorizeStartAndResourceIncreaseRequest(
+          NMTokenIdentifier nmTokenIdentifier,
+          ContainerTokenIdentifier containerTokenIdentifier,
+          boolean startRequest) throws YarnException {
+        // do nothing
+      }
+
       @Override
         protected void updateNMTokenIdentifier(
             NMTokenIdentifier nmTokenIdentifier) throws InvalidToken {
@@ -310,4 +311,34 @@ public abstract class BaseContainerManagerTest {
         app.getApplicationState().equals(finalState));
   }
 
+  public static void waitForNMContainerState(ContainerManagerImpl
+      containerManager, ContainerId containerID,
+          org.apache.hadoop.yarn.server.nodemanager.containermanager
+              .container.ContainerState finalState)
+                  throws InterruptedException, YarnException, IOException {
+    waitForNMContainerState(containerManager, containerID, finalState, 20);
+  }
+
+  public static void waitForNMContainerState(ContainerManagerImpl
+      containerManager, ContainerId containerID,
+          org.apache.hadoop.yarn.server.nodemanager.containermanager
+          .container.ContainerState finalState, int timeOutMax)
+              throws InterruptedException, YarnException, IOException {
+    Container container =
+        containerManager.getContext().getContainers().get(containerID);
+    org.apache.hadoop.yarn.server.nodemanager
+        .containermanager.container.ContainerState currentState =
+            container.getContainerState();
+    int timeoutSecs = 0;
+    while (!currentState.equals(finalState)
+        && timeoutSecs++ < timeOutMax) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for NM container to get into state " + finalState
+          + ". Current state is " + currentState);
+      currentState = container.getContainerState();
+    }
+    LOG.info("Container state is " + currentState);
+    Assert.assertEquals("ContainerState is not correct (timedout)",
+        finalState, currentState);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b05443ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index e508424..e2f12ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -72,6 +74,7 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
+import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA;
@@ -87,6 +90,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import static org.junit.Assert.assertEquals;
+
 public class TestContainerManager extends BaseContainerManagerTest {
 
   public TestContainerManager() throws UnsupportedFileSystemException {
@@ -803,7 +808,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
         metrics, dirsHandler);
     String strExceptionMsg = "";
     try {
-      cMgrImpl.authorizeStartRequest(null, new ContainerTokenIdentifier());
+      cMgrImpl.authorizeStartAndResourceIncreaseRequest(
+          null, new ContainerTokenIdentifier(), true);
     } catch(YarnException ye) {
       strExceptionMsg = ye.getMessage();
     }
@@ -812,7 +818,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     strExceptionMsg = "";
     try {
-      cMgrImpl.authorizeStartRequest(new NMTokenIdentifier(), null);
+      cMgrImpl.authorizeStartAndResourceIncreaseRequest(
+          new NMTokenIdentifier(), null, true);
     } catch(YarnException ye) {
       strExceptionMsg = ye.getMessage();
     }
@@ -878,6 +885,167 @@ public class TestContainerManager extends BaseContainerManagerTest {
         ContainerManagerImpl.INVALID_CONTAINERTOKEN_MSG);
   }
 
+  @Test
+  public void testIncreaseContainerResourceWithInvalidRequests() throws Exception {
+    containerManager.start();
+    // Start 4 containers 0..4 with default resource (1024, 1)
+    List<StartContainerRequest> list = new ArrayList<>();
+    ContainerLaunchContext containerLaunchContext = recordFactory
+        .newRecordInstance(ContainerLaunchContext.class);
+    for (int i = 0; i < 4; i++) {
+      ContainerId cId = createContainerId(i);
+      long identifier = DUMMY_RM_IDENTIFIER;
+      Token containerToken = createContainerToken(cId, identifier,
+          context.getNodeId(), user, context.getContainerTokenSecretManager());
+      StartContainerRequest request = StartContainerRequest.newInstance(
+          containerLaunchContext, containerToken);
+      list.add(request);
+    }
+    StartContainersRequest requestList = StartContainersRequest
+        .newInstance(list);
+    StartContainersResponse response = containerManager
+        .startContainers(requestList);
+
+    Assert.assertEquals(4, response.getSuccessfullyStartedContainers().size());
+    int i = 0;
+    for (ContainerId id : response.getSuccessfullyStartedContainers()) {
+      Assert.assertEquals(i, id.getContainerId());
+      i++;
+    }
+
+    Thread.sleep(2000);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request for container-0, the request will fail as the
+    // container will have exited, and won't be in RUNNING state
+    ContainerId cId0 = createContainerId(0);
+    Token containerToken =
+        createContainerToken(cId0, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+                Resource.newInstance(1234, 3),
+                    context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    // Add increase request for container-7, the request will fail as the
+    // container does not exist
+    ContainerId cId7 = createContainerId(7);
+    containerToken =
+        createContainerToken(cId7, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+            Resource.newInstance(1234, 3),
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest
+          .newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    // Check response
+    Assert.assertEquals(
+        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertEquals(2, increaseResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        .getFailedRequests().entrySet()) {
+      Assert.assertNotNull("Failed message", entry.getValue().getMessage());
+      if (cId0.equals(entry.getKey())) {
+        Assert.assertTrue(entry.getValue().getMessage()
+          .contains("Resource can only be changed when a "
+              + "container is in RUNNING state"));
+      } else if (cId7.equals(entry.getKey())) {
+        Assert.assertTrue(entry.getValue().getMessage()
+            .contains("Container " + cId7.toString()
+                + " is not handled by this NodeManager"));
+      } else {
+        throw new YarnException("Received failed request from wrong"
+            + " container: " + entry.getKey().toString());
+      }
+    }
+  }
+
+  @Test
+  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
+    containerManager.start();
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha =
+        recordFactory.newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, rsrc_alpha);
+    containerLaunchContext.setLocalResources(localResources);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    containerLaunchContext.setCommands(commands);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request. The increase request should fail
+    // as the current resource does not fit in the target resource
+    Token containerToken =
+        createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+            Resource.newInstance(512, 1),
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest
+            .newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    // Check response
+    Assert.assertEquals(
+        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertEquals(1, increaseResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        .getFailedRequests().entrySet()) {
+      if (cId.equals(entry.getKey())) {
+        Assert.assertNotNull("Failed message", entry.getValue().getMessage());
+        Assert.assertTrue(entry.getValue().getMessage()
+            .contains("The target resource "
+                + Resource.newInstance(512, 1).toString()
+                + " is smaller than the current resource "
+                + Resource.newInstance(1024, 1)));
+      } else {
+        throw new YarnException("Received failed request from wrong"
+            + " container: " + entry.getKey().toString());
+      }
+    }
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)
@@ -892,15 +1060,21 @@ public class TestContainerManager extends BaseContainerManagerTest {
       LogAggregationContext logAggregationContext)
       throws IOException {
     Resource r = BuilderUtils.newResource(1024, 1);
+    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
+        containerTokenSecretManager, logAggregationContext);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
     ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, r,
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
           System.currentTimeMillis() + 100000L, 123, rmIdentifier,
           Priority.newInstance(0), 0, logAggregationContext, null);
-    Token containerToken =
-        BuilderUtils
-          .newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+        .retrievePassword(containerTokenIdentifier),
             containerTokenIdentifier);
-    return containerToken;
   }
 }


[09/50] [abbrv] hadoop git commit: HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe)

Posted by wa...@apache.org.
HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe)


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

Branch: refs/heads/YARN-1197
Commit: 24f6a7c9563757234f53ca23e12f9c9208b53082
Parents: 8fa41d9
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Aug 31 17:31:29 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Aug 31 18:06:30 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../bkjournal/BookKeeperEditLogInputStream.java |   2 +-
 .../hadoop/hdfs/protocol/LayoutVersion.java     |   2 +-
 .../namenode/EditLogBackupInputStream.java      |   2 +-
 .../server/namenode/EditLogFileInputStream.java |   2 +-
 .../hdfs/server/namenode/FSEditLogOp.java       | 354 +++++++++++++------
 .../hdfs/server/namenode/TestEditLog.java       |   2 +-
 .../namenode/TestEditLogFileInputStream.java    |  80 +++++
 8 files changed, 341 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6584c84..57ddcb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -872,6 +872,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8946. Improve choosing datanode storage for block placement. (yliu)
 
+    HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
index e2098dd..86da807 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
@@ -83,7 +83,7 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
     tracker = new FSEditLogLoader.PositionTrackingInputStream(bin);
     DataInputStream in = new DataInputStream(tracker);
 
-    reader = new FSEditLogOp.Reader(in, tracker, logVersion);
+    reader = FSEditLogOp.Reader.create(in, tracker, logVersion);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
index c893744..1750790 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
@@ -87,7 +87,7 @@ public class LayoutVersion {
     FSIMAGE_COMPRESSION(-25, "Support for fsimage compression"),
     FSIMAGE_CHECKSUM(-26, "Support checksum for fsimage"),
     REMOVE_REL13_DISK_LAYOUT_SUPPORT(-27, "Remove support for 0.13 disk layout"),
-    EDITS_CHESKUM(-28, "Support checksum for editlog"),
+    EDITS_CHECKSUM(-28, "Support checksum for editlog"),
     UNUSED(-29, "Skipped version"),
     FSIMAGE_NAME_OPTIMIZATION(-30, "Store only last part of path in fsimage"),
     RESERVED_REL20_203(-31, -19, "Reserved for release 0.20.203", true,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
index 689cacc..81d285a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
@@ -119,7 +119,7 @@ class EditLogBackupInputStream extends EditLogInputStream {
 
     this.version = version;
 
-    reader = new FSEditLogOp.Reader(in, tracker, version);
+    reader = FSEditLogOp.Reader.create(in, tracker, version);
   }
 
   void clear() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 3e21c24..73a162e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -157,7 +157,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
               "flags from log");
         }
       }
-      reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
+      reader = FSEditLogOp.Reader.create(dataIn, tracker, logVersion);
       reader.setMaxOpSize(maxOpSize);
       state = State.OPEN;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index ab36f17..125e1cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -4518,42 +4518,46 @@ public abstract class FSEditLogOp {
   /**
    * Class for reading editlog ops from a stream
    */
-  public static class Reader {
-    private final DataInputStream in;
-    private final StreamLimiter limiter;
-    private final int logVersion;
-    private final Checksum checksum;
-    private final OpInstanceCache cache;
-    private int maxOpSize;
-    private final boolean supportEditLogLength;
+  public abstract static class Reader {
+    final DataInputStream in;
+    final StreamLimiter limiter;
+    final OpInstanceCache cache;
+    final byte[] temp = new byte[4096];
+    final int logVersion;
+    int maxOpSize;
+
+    public static Reader create(DataInputStream in, StreamLimiter limiter,
+                                int logVersion) {
+      if (logVersion < NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION) {
+        // Use the LengthPrefixedReader on edit logs which are newer than what
+        // we can parse.  (Newer layout versions are represented by smaller
+        // negative integers, for historical reasons.) Even though we can't
+        // parse the Ops contained in them, we should still be able to call
+        // scanOp on them.  This is important for the JournalNode during rolling
+        // upgrade.
+        return new LengthPrefixedReader(in, limiter, logVersion);
+      } else if (NameNodeLayoutVersion.supports(
+              NameNodeLayoutVersion.Feature.EDITLOG_LENGTH, logVersion)) {
+        return new LengthPrefixedReader(in, limiter, logVersion);
+      } else if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITS_CHECKSUM, logVersion)) {
+        Checksum checksum = DataChecksum.newCrc32();
+        return new ChecksummedReader(checksum, in, limiter, logVersion);
+      } else {
+        return new LegacyReader(in, limiter, logVersion);
+      }
+    }
 
     /**
      * Construct the reader
-     * @param in The stream to read from.
-     * @param logVersion The version of the data coming from the stream.
+     * @param in            The stream to read from.
+     * @param limiter       The limiter for this stream.
+     * @param logVersion    The version of the data coming from the stream.
      */
-    public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
-      this.logVersion = logVersion;
-      if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
-        this.checksum = DataChecksum.newCrc32();
-      } else {
-        this.checksum = null;
-      }
-      // It is possible that the logVersion is actually a future layoutversion
-      // during the rolling upgrade (e.g., the NN gets upgraded first). We
-      // assume future layout will also support length of editlog op.
-      this.supportEditLogLength = NameNodeLayoutVersion.supports(
-          NameNodeLayoutVersion.Feature.EDITLOG_LENGTH, logVersion)
-          || logVersion < NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
-
-      if (this.checksum != null) {
-        this.in = new DataInputStream(
-            new CheckedInputStream(in, this.checksum));
-      } else {
-        this.in = in;
-      }
+    Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
+      this.in = in;
       this.limiter = limiter;
+      this.logVersion = logVersion;
       this.cache = new OpInstanceCache();
       this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
     }
@@ -4606,26 +4610,25 @@ public abstract class FSEditLogOp {
       }
     }
 
-    private void verifyTerminator() throws IOException {
+    void verifyTerminator() throws IOException {
       /** The end of the edit log should contain only 0x00 or 0xff bytes.
        * If it contains other bytes, the log itself may be corrupt.
        * It is important to check this; if we don't, a stray OP_INVALID byte 
        * could make us stop reading the edit log halfway through, and we'd never
        * know that we had lost data.
        */
-      byte[] buf = new byte[4096];
       limiter.clearLimit();
       int numRead = -1, idx = 0;
       while (true) {
         try {
           numRead = -1;
           idx = 0;
-          numRead = in.read(buf);
+          numRead = in.read(temp);
           if (numRead == -1) {
             return;
           }
           while (idx < numRead) {
-            if ((buf[idx] != (byte)0) && (buf[idx] != (byte)-1)) {
+            if ((temp[idx] != (byte)0) && (temp[idx] != (byte)-1)) {
               throw new IOException("Read extra bytes after " +
                 "the terminator!");
             }
@@ -4638,7 +4641,7 @@ public abstract class FSEditLogOp {
           if (numRead != -1) { 
             in.reset();
             IOUtils.skipFully(in, idx);
-            in.mark(buf.length + 1);
+            in.mark(temp.length + 1);
             IOUtils.skipFully(in, 1);
           }
         }
@@ -4653,14 +4656,164 @@ public abstract class FSEditLogOp {
      * If an exception is thrown, the stream's mark will be set to the first
      * problematic byte.  This usually means the beginning of the opcode.
      */
-    private FSEditLogOp decodeOp() throws IOException {
-      limiter.setLimit(maxOpSize);
+    public abstract FSEditLogOp decodeOp() throws IOException;
+
+    /**
+     * Similar to decodeOp(), but we only retrieve the transaction ID of the
+     * Op rather than reading it.  If the edit log format supports length
+     * prefixing, this can be much faster than full decoding.
+     *
+     * @return the last txid of the segment, or INVALID_TXID on EOF.
+     */
+    public abstract long scanOp() throws IOException;
+  }
+
+  /**
+   * Reads edit logs which are prefixed with a length.  These edit logs also
+   * include a checksum and transaction ID.
+   */
+  private static class LengthPrefixedReader extends Reader {
+    /**
+     * The minimum length of a length-prefixed Op.
+     *
+     * The minimum Op has:
+     * 1-byte opcode
+     * 4-byte length
+     * 8-byte txid
+     * 0-byte body
+     * 4-byte checksum
+     */
+    private static final int MIN_OP_LENGTH = 17;
+
+    /**
+     * The op id length.
+     *
+     * Not included in the stored length.
+     */
+    private static final int OP_ID_LENGTH = 1;
+
+    /**
+     * The checksum length.
+     *
+     * Not included in the stored length.
+     */
+    private static final int CHECKSUM_LENGTH = 4;
+
+    private final Checksum checksum;
+
+    LengthPrefixedReader(DataInputStream in, StreamLimiter limiter,
+                         int logVersion) {
+      super(in, limiter, logVersion);
+      this.checksum = DataChecksum.newCrc32();
+    }
+
+    @Override
+    public FSEditLogOp decodeOp() throws IOException {
+      long txid = decodeOpFrame();
+      if (txid == HdfsServerConstants.INVALID_TXID) {
+        return null;
+      }
+      in.reset();
       in.mark(maxOpSize);
+      FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(in.readByte());
+      FSEditLogOp op = cache.get(opCode);
+      if (op == null) {
+        throw new IOException("Read invalid opcode " + opCode);
+      }
+      op.setTransactionId(txid);
+      IOUtils.skipFully(in, 4 + 8); // skip length and txid
+      op.readFields(in, logVersion);
+      // skip over the checksum, which we validated above.
+      IOUtils.skipFully(in, CHECKSUM_LENGTH);
+      return op;
+    }
+
+    @Override
+    public long scanOp() throws IOException {
+      return decodeOpFrame();
+    }
 
-      if (checksum != null) {
-        checksum.reset();
+    /**
+     * Decode the opcode "frame".  This includes reading the opcode and
+     * transaction ID, and validating the checksum and length.  It does not
+     * include reading the opcode-specific fields.
+     * The input stream will be advanced to the end of the op at the end of this
+     * function.
+     *
+     * @return        An op with the txid set, but none of the other fields
+     *                  filled in, or null if we hit EOF.
+     */
+    private long decodeOpFrame() throws IOException {
+      limiter.setLimit(maxOpSize);
+      in.mark(maxOpSize);
+      byte opCodeByte;
+      try {
+        opCodeByte = in.readByte();
+      } catch (EOFException eof) {
+        // EOF at an opcode boundary is expected.
+        return HdfsServerConstants.INVALID_TXID;
       }
+      if (opCodeByte == FSEditLogOpCodes.OP_INVALID.getOpCode()) {
+        verifyTerminator();
+        return HdfsServerConstants.INVALID_TXID;
+      }
+      // Here, we verify that the Op size makes sense and that the
+      // data matches its checksum before attempting to construct an Op.
+      // This is important because otherwise we may encounter an
+      // OutOfMemoryException which could bring down the NameNode or
+      // JournalNode when reading garbage data.
+      int opLength =  in.readInt() + OP_ID_LENGTH + CHECKSUM_LENGTH;
+      if (opLength > maxOpSize) {
+        throw new IOException("Op " + (int)opCodeByte + " has size " +
+            opLength + ", but maxOpSize = " + maxOpSize);
+      } else  if (opLength < MIN_OP_LENGTH) {
+        throw new IOException("Op " + (int)opCodeByte + " has size " +
+            opLength + ", but the minimum op size is " + MIN_OP_LENGTH);
+      }
+      long txid = in.readLong();
+      // Verify checksum
+      in.reset();
+      in.mark(maxOpSize);
+      checksum.reset();
+      for (int rem = opLength - CHECKSUM_LENGTH; rem > 0;) {
+        int toRead = Math.min(temp.length, rem);
+        IOUtils.readFully(in, temp, 0, toRead);
+        checksum.update(temp, 0, toRead);
+        rem -= toRead;
+      }
+      int expectedChecksum = in.readInt();
+      int calculatedChecksum = (int)checksum.getValue();
+      if (expectedChecksum != calculatedChecksum) {
+        throw new ChecksumException(
+            "Transaction is corrupt. Calculated checksum is " +
+            calculatedChecksum + " but read checksum " +
+            expectedChecksum, txid);
+      }
+      return txid;
+    }
+  }
+
+  /**
+   * Read edit logs which have a checksum and a transaction ID, but not a
+   * length.
+   */
+  private static class ChecksummedReader extends Reader {
+    private final Checksum checksum;
 
+    ChecksummedReader(Checksum checksum, DataInputStream in,
+                      StreamLimiter limiter, int logVersion) {
+      super(new DataInputStream(
+          new CheckedInputStream(in, checksum)), limiter, logVersion);
+      this.checksum = checksum;
+    }
+
+    @Override
+    public FSEditLogOp decodeOp() throws IOException {
+      limiter.setLimit(maxOpSize);
+      in.mark(maxOpSize);
+      // Reset the checksum.  Since we are using a CheckedInputStream, each
+      // subsequent read from the  stream will update the checksum.
+      checksum.reset();
       byte opCodeByte;
       try {
         opCodeByte = in.readByte();
@@ -4668,88 +4821,89 @@ public abstract class FSEditLogOp {
         // EOF at an opcode boundary is expected.
         return null;
       }
-
       FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
       if (opCode == OP_INVALID) {
         verifyTerminator();
         return null;
       }
-
       FSEditLogOp op = cache.get(opCode);
       if (op == null) {
         throw new IOException("Read invalid opcode " + opCode);
       }
-
-      if (supportEditLogLength) {
-        in.readInt();
+      op.setTransactionId(in.readLong());
+      op.readFields(in, logVersion);
+      // Verify checksum
+      int calculatedChecksum = (int)checksum.getValue();
+      int expectedChecksum = in.readInt();
+      if (expectedChecksum != calculatedChecksum) {
+        throw new ChecksumException(
+            "Transaction is corrupt. Calculated checksum is " +
+                calculatedChecksum + " but read checksum " +
+                expectedChecksum, op.txid);
       }
+      return op;
+    }
 
+    @Override
+    public long scanOp() throws IOException {
+      // Edit logs of this age don't have any length prefix, so we just have
+      // to read the entire Op.
+      FSEditLogOp op = decodeOp();
+      return op == null ?
+          HdfsServerConstants.INVALID_TXID : op.getTransactionId();
+    }
+  }
+
+  /**
+   * Read older edit logs which may or may not have transaction IDs and other
+   * features.  This code is used during upgrades and to allow HDFS INotify to
+   * read older edit log files.
+   */
+  private static class LegacyReader extends Reader {
+    LegacyReader(DataInputStream in,
+                      StreamLimiter limiter, int logVersion) {
+      super(in, limiter, logVersion);
+    }
+
+    @Override
+    public FSEditLogOp decodeOp() throws IOException {
+      limiter.setLimit(maxOpSize);
+      in.mark(maxOpSize);
+      byte opCodeByte;
+      try {
+        opCodeByte = in.readByte();
+      } catch (EOFException eof) {
+        // EOF at an opcode boundary is expected.
+        return null;
+      }
+      FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
+      if (opCode == OP_INVALID) {
+        verifyTerminator();
+        return null;
+      }
+      FSEditLogOp op = cache.get(opCode);
+      if (op == null) {
+        throw new IOException("Read invalid opcode " + opCode);
+      }
       if (NameNodeLayoutVersion.supports(
-          LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
-        // Read the txid
+            LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
         op.setTransactionId(in.readLong());
       } else {
         op.setTransactionId(HdfsServerConstants.INVALID_TXID);
       }
-
       op.readFields(in, logVersion);
-
-      validateChecksum(in, checksum, op.txid);
       return op;
     }
 
-    /**
-     * Similar with decodeOp(), but instead of doing the real decoding, we skip
-     * the content of the op if the length of the editlog is supported.
-     * @return the last txid of the segment, or INVALID_TXID on exception
-     */
+    @Override
     public long scanOp() throws IOException {
-      if (supportEditLogLength) {
-        limiter.setLimit(maxOpSize);
-        in.mark(maxOpSize);
-
-        final byte opCodeByte;
-        try {
-          opCodeByte = in.readByte(); // op code
-        } catch (EOFException e) {
-          return HdfsServerConstants.INVALID_TXID;
-        }
-
-        FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
-        if (opCode == OP_INVALID) {
-          verifyTerminator();
-          return HdfsServerConstants.INVALID_TXID;
-        }
-
-        int length = in.readInt(); // read the length of the op
-        long txid = in.readLong(); // read the txid
-
-        // skip the remaining content
-        IOUtils.skipFully(in, length - 8); 
-        // TODO: do we want to verify checksum for JN? For now we don't.
-        return txid;
-      } else {
-        FSEditLogOp op = decodeOp();
-        return op == null ? HdfsServerConstants.INVALID_TXID : op.getTransactionId();
-      }
-    }
-
-    /**
-     * Validate a transaction's checksum
-     */
-    private void validateChecksum(DataInputStream in,
-                                  Checksum checksum,
-                                  long txid)
-        throws IOException {
-      if (checksum != null) {
-        int calculatedChecksum = (int)checksum.getValue();
-        int readChecksum = in.readInt(); // read in checksum
-        if (readChecksum != calculatedChecksum) {
-          throw new ChecksumException(
-              "Transaction is corrupt. Calculated checksum is " +
-              calculatedChecksum + " but read checksum " + readChecksum, txid);
-        }
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
+        throw new IOException("Can't scan a pre-transactional edit log.");
       }
+      FSEditLogOp op = decodeOp();
+      return op == null ?
+          HdfsServerConstants.INVALID_TXID : op.getTransactionId();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 68d008f..e59dec4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -875,7 +875,7 @@ public class TestEditLog {
       tracker = new FSEditLogLoader.PositionTrackingInputStream(in);
       in = new DataInputStream(tracker);
             
-      reader = new FSEditLogOp.Reader(in, tracker, version);
+      reader = FSEditLogOp.Reader.create(in, tracker, version);
     }
   
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24f6a7c9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
index c0eb890..aecdc78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
@@ -25,19 +25,35 @@ import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 
 import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.EnumMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
 
 public class TestEditLogFileInputStream {
+  private static final Log LOG =
+      LogFactory.getLog(TestEditLogFileInputStream.class);
   private static final byte[] FAKE_LOG_DATA = TestEditLog.HADOOP20_SOME_EDITS;
 
+  private final static File TEST_DIR = PathUtils
+      .getTestDir(TestEditLogFileInputStream.class);
+
   @Test
   public void testReadURL() throws Exception {
     HttpURLConnection conn = mock(HttpURLConnection.class);
@@ -63,4 +79,68 @@ public class TestEditLogFileInputStream {
     assertEquals(FAKE_LOG_DATA.length, elis.length());
     elis.close();
   }
+
+  /**
+   * Regression test for HDFS-8965 which verifies that
+   * FSEditLogFileInputStream#scanOp verifies Op checksums.
+   */
+  @Test(timeout=60000)
+  public void testScanCorruptEditLog() throws Exception {
+    Configuration conf = new Configuration();
+    File editLog = new File(System.getProperty(
+        "test.build.data", "/tmp"), "testCorruptEditLog");
+
+    LOG.debug("Creating test edit log file: " + editLog);
+    EditLogFileOutputStream elos = new EditLogFileOutputStream(conf,
+        editLog.getAbsoluteFile(), 8192);
+    elos.create(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
+    FSEditLogOp.OpInstanceCache cache = new FSEditLogOp.OpInstanceCache();
+    FSEditLogOp.MkdirOp mkdirOp = FSEditLogOp.MkdirOp.getInstance(cache);
+    mkdirOp.reset();
+    mkdirOp.setRpcCallId(123);
+    mkdirOp.setTransactionId(1);
+    mkdirOp.setInodeId(789L);
+    mkdirOp.setPath("/mydir");
+    PermissionStatus perms = PermissionStatus.createImmutable(
+        "myuser", "mygroup", FsPermission.createImmutable((short)0777));
+    mkdirOp.setPermissionStatus(perms);
+    elos.write(mkdirOp);
+    mkdirOp.reset();
+    mkdirOp.setRpcCallId(456);
+    mkdirOp.setTransactionId(2);
+    mkdirOp.setInodeId(123L);
+    mkdirOp.setPath("/mydir2");
+    perms = PermissionStatus.createImmutable(
+        "myuser", "mygroup", FsPermission.createImmutable((short)0666));
+    mkdirOp.setPermissionStatus(perms);
+    elos.write(mkdirOp);
+    elos.setReadyToFlush();
+    elos.flushAndSync(false);
+    elos.close();
+    long fileLen = editLog.length();
+
+    LOG.debug("Corrupting last 4 bytes of edit log file " + editLog +
+        ", whose length is " + fileLen);
+    RandomAccessFile rwf = new RandomAccessFile(editLog, "rw");
+    rwf.seek(fileLen - 4);
+    int b = rwf.readInt();
+    rwf.seek(fileLen - 4);
+    rwf.writeInt(b + 1);
+    rwf.close();
+
+    EditLogFileInputStream elis = new EditLogFileInputStream(editLog);
+    Assert.assertEquals(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION,
+        elis.getVersion(true));
+    Assert.assertEquals(1, elis.scanNextOp());
+    LOG.debug("Read transaction 1 from " + editLog);
+    try {
+      elis.scanNextOp();
+      Assert.fail("Expected scanNextOp to fail when op checksum was corrupt.");
+    } catch (IOException e) {
+      LOG.debug("Caught expected checksum error when reading corrupt " +
+          "transaction 2", e);
+      GenericTestUtils.assertExceptionContains("Transaction is corrupt.", e);
+    }
+    elis.close();
+  }
 }


[23/50] [abbrv] hadoop git commit: HDFS-2070. Add more unit tests for FsShell getmerge (Daniel Templeton via Colin P. McCabe)

Posted by wa...@apache.org.
HDFS-2070. Add more unit tests for FsShell getmerge (Daniel Templeton via Colin P. McCabe)


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

Branch: refs/heads/YARN-1197
Commit: c006c3a1e6d3631c5724e1eb0a0b620d9f7c8dcc
Parents: 7d6687f
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Sep 2 12:36:14 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Sep 2 12:36:14 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../src/test/resources/testHDFSConf.xml         | 142 ++++++++++++++++---
 2 files changed, 123 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c006c3a1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 14a9248..0f2d713 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -874,6 +874,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8965. Harden edit log reading code against out of memory errors (cmccabe)
 
+    HDFS-2070. Add more unit tests for FsShell getmerge (Daniel Templeton via
+    Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c006c3a1/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 4e25d09..18c68ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -5268,7 +5268,126 @@
     </test>
     
     <!-- Tests for getmerge -->
-    <!-- Manual Testing -->
+    <test> <!-- TESTED -->
+      <description>getmerge: basic</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
+        <command>rm data</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output>12345678901234
+12345678901234
+12345678901234
+</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>getmerge: with file</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1/data15bytes CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
+        <command>rm data</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output>12345678901234
+</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>getmerge: with multiple files</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1/data15bytes /user/USERNAME/dir1/data30bytes CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
+        <command>rm data</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output>12345678901234
+12345678901234
+12345678901234
+</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>getmerge: with newlines</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -getmerge -nl /user/USERNAME/dir1 CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
+        <command>rm data</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output>12345678901234
+
+12345678901234
+12345678901234
+
+</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>getmerge: with multiple files and newlines</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
+        <command>-fs NAMENODE -getmerge -nl /user/USERNAME/dir1/data15bytes /user/USERNAME/dir1/data30bytes CLITEST_DATA/file</command>
+        <command>-cat CLITEST_DATA/file</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
+        <command>rm data</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output>12345678901234
+
+12345678901234
+12345678901234
+
+</expected-output>
+        </comparator>
+      </comparators>
+    </test>
 
     <!-- Tests for cat -->
     <test> <!-- TESTED -->
@@ -16515,27 +16634,6 @@
       </comparators>
     </test>
 
-    <test>
-      <description>getmerge</description>
-      <test-commands>
-        <command>-fs NAMENODE -mkdir -p /user/USERNAME/dir1</command>
-        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data15bytes /user/USERNAME/dir1</command>
-        <command>-fs NAMENODE -copyFromLocal CLITEST_DATA/data30bytes /user/USERNAME/dir1</command>
-        <command>-fs NAMENODE -getmerge /user/USERNAME/dir1 CLITEST_DATA/file</command>
-        <command>-cat CLITEST_DATA/file</command>
-      </test-commands>
-      <cleanup-commands>
-        <command>-fs NAMENODE -rm -r /user/USERNAME CLITEST_DATA/file</command>
-        <command>rm data</command>
-      </cleanup-commands>
-      <comparators>
-        <comparator>
-          <type>RegexpComparator</type>
-          <expected-output>12345678901234.*</expected-output>
-        </comparator>
-      </comparators>
-    </test>
-
     <!-- Tests for snapshots -->
     <test>
       <description>allowSnapshot</description>


[28/50] [abbrv] hadoop git commit: HDFS-8885. ByteRangeInputStream used in webhdfs does not override available(). Contributed by Shradha Revankar.

Posted by wa...@apache.org.
HDFS-8885. ByteRangeInputStream used in webhdfs does not override available(). Contributed by Shradha Revankar.


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

Branch: refs/heads/YARN-1197
Commit: c92e31bd659e95c8baa0f3b2bf0cd7f6f72278e6
Parents: 09c64ba
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Sep 3 19:40:34 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Sep 3 19:40:34 2015 +0900

----------------------------------------------------------------------
 .../hadoop/hdfs/web/ByteRangeInputStream.java   | 11 +++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/web/TestByteRangeInputStream.java      | 79 ++++++++++++++++++++
 3 files changed, 93 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c92e31bd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java
index bb581db..911d6a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/ByteRangeInputStream.java
@@ -274,4 +274,15 @@ public abstract class ByteRangeInputStream extends FSInputStream {
     }
     status = StreamStatus.CLOSED;
   }
+
+  @Override
+  public synchronized int available() throws IOException{
+    getInputStream();
+    if(fileLength != null){
+      long remaining = fileLength - currentPos;
+      return remaining <= Integer.MAX_VALUE ? (int) remaining : Integer.MAX_VALUE;
+    }else {
+      return Integer.MAX_VALUE;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c92e31bd/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e68c011..22b80a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1284,6 +1284,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9003. ForkJoin thread pool leaks. (Kihwal Lee via jing9)
 
+    HDFS-8885. ByteRangeInputStream used in webhdfs does not override
+    available(). (Shradha Revankar via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c92e31bd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
index 40f2b9c..7f1f00f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
@@ -210,4 +210,83 @@ public class TestByteRangeInputStream {
 
     verify(mockStream.in, times(isCloses)).close();
   }
+
+
+  @Test
+  public void testAvailable() throws IOException {
+    ByteRangeInputStream bris =
+            mock(ByteRangeInputStream.class, CALLS_REAL_METHODS);
+    InputStreamAndFileLength mockStream = new InputStreamAndFileLength(65535L,
+            mock(InputStream.class));
+    doReturn(mockStream).when(bris).openInputStream(Mockito.anyLong());
+    Whitebox.setInternalState(bris, "status",
+            ByteRangeInputStream.StreamStatus.SEEK);
+
+
+    assertEquals("Before read or seek, available should be same as filelength",
+            65535, bris.available());
+    verify(bris, times(1)).openInputStream(Mockito.anyLong());
+
+    bris.seek(10);
+    assertEquals("Seek 10 bytes, available should return filelength - 10"
+            , 65525,
+            bris.available());
+
+    //no more bytes available
+    bris.seek(65535);
+    assertEquals("Seek till end of file, available should return 0 bytes", 0,
+            bris.available());
+
+    //test reads, seek back to 0 and start reading
+    bris.seek(0);
+    bris.read();
+    assertEquals("Read 1 byte, available must return  filelength - 1",
+            65534, bris.available());
+
+    bris.read();
+    assertEquals("Read another 1 byte, available must return  filelength - 2",
+            65533, bris.available());
+
+    //seek and read
+    bris.seek(100);
+    bris.read();
+    assertEquals("Seek to offset 100 and read 1 byte, available should return filelength - 101",
+            65434, bris.available());
+    bris.close();
+  }
+
+  @Test
+  public void testAvailableLengthNotKnown() throws IOException {
+    ByteRangeInputStream bris =
+            mock(ByteRangeInputStream.class, CALLS_REAL_METHODS);
+    //Length is null for chunked transfer-encoding
+    InputStreamAndFileLength mockStream = new InputStreamAndFileLength(null,
+            mock(InputStream.class));
+    doReturn(mockStream).when(bris).openInputStream(Mockito.anyLong());
+    Whitebox.setInternalState(bris, "status",
+            ByteRangeInputStream.StreamStatus.SEEK);
+
+    assertEquals(Integer.MAX_VALUE, bris.available());
+  }
+
+  @Test
+  public void testAvailableStreamClosed() throws IOException {
+    ByteRangeInputStream bris =
+            mock(ByteRangeInputStream.class, CALLS_REAL_METHODS);
+    InputStreamAndFileLength mockStream = new InputStreamAndFileLength(null,
+            mock(InputStream.class));
+    doReturn(mockStream).when(bris).openInputStream(Mockito.anyLong());
+    Whitebox.setInternalState(bris, "status",
+            ByteRangeInputStream.StreamStatus.SEEK);
+
+    bris.close();
+    try{
+      bris.available();
+      fail("Exception should be thrown when stream is closed");
+    }catch(IOException e){
+      assertTrue("Exception when stream is closed",
+              e.getMessage().equals("Stream closed"));
+    }
+  }
+
 }


[50/50] [abbrv] hadoop git commit: YARN-3868. Recovery support for container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3868. Recovery support for container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: f86eae17d3bb8e25fee7d41c65851a7bf85e3610
Parents: cf3d5e5
Author: Jian He <ji...@apache.org>
Authored: Thu Aug 20 21:18:23 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:50:45 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../containermanager/ContainerManagerImpl.java  |   5 +-
 .../container/ContainerImpl.java                |   8 +-
 .../recovery/NMLeveldbStateStoreService.java    |  22 ++
 .../recovery/NMNullStateStoreService.java       |   6 +
 .../recovery/NMStateStoreService.java           |  15 ++
 .../nodemanager/TestNodeManagerResync.java      |   2 +-
 .../TestContainerManagerRecovery.java           | 233 ++++++++++++++++++-
 .../recovery/NMMemoryStateStoreService.java     |  11 +-
 .../TestNMLeveldbStateStoreService.java         |  11 +
 10 files changed, 302 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 429fa12..d9c0071 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -208,6 +208,8 @@ Release 2.8.0 - UNRELEASED
     YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to
     support container resizing. (Meng Ding via jianhe)
 
+    YARN-3868. Recovery support for container resizing. (Meng Ding via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 76d87b9..298cf98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -324,7 +324,7 @@ public class ContainerManagerImpl extends CompositeService implements
       Container container = new ContainerImpl(getConfig(), dispatcher,
           context.getNMStateStore(), req.getContainerLaunchContext(),
           credentials, metrics, token, rcs.getStatus(), rcs.getExitCode(),
-          rcs.getDiagnostics(), rcs.getKilled());
+          rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability());
       context.getContainers().put(containerId, container);
       dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
@@ -1068,6 +1068,9 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
+        // Persist container resource change for recovery
+        this.context.getNMStateStore().storeContainerResourceChanged(
+            containerId, targetResource);
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(
                 containerId, targetResource));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 5c61a92..eff2188 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -154,13 +154,19 @@ public class ContainerImpl implements Container {
       Credentials creds, NodeManagerMetrics metrics,
       ContainerTokenIdentifier containerTokenIdentifier,
       RecoveredContainerStatus recoveredStatus, int exitCode,
-      String diagnostics, boolean wasKilled) {
+      String diagnostics, boolean wasKilled, Resource recoveredCapability) {
     this(conf, dispatcher, stateStore, launchContext, creds, metrics,
         containerTokenIdentifier);
     this.recoveredStatus = recoveredStatus;
     this.exitCode = exitCode;
     this.recoveredAsKilled = wasKilled;
     this.diagnostics.append(diagnostics);
+    if (recoveredCapability != null
+        && !this.resource.equals(recoveredCapability)) {
+      // resource capability had been updated before NM was down
+      this.resource = Resource.newInstance(recoveredCapability.getMemory(),
+          recoveredCapability.getVirtualCores());
+    }
   }
 
   private static final ContainerDiagnosticsUpdateTransition UPDATE_DIAGNOSTICS_TRANSITION =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index df58182..89c71bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -40,7 +40,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestP
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
@@ -99,6 +102,8 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request";
   private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics";
   private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched";
+  private static final String CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX =
+      "/resourceChanged";
   private static final String CONTAINER_KILLED_KEY_SUFFIX = "/killed";
   private static final String CONTAINER_EXIT_CODE_KEY_SUFFIX = "/exitcode";
 
@@ -230,6 +235,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       } else if (suffix.equals(CONTAINER_EXIT_CODE_KEY_SUFFIX)) {
         rcs.status = RecoveredContainerStatus.COMPLETED;
         rcs.exitCode = Integer.parseInt(asString(entry.getValue()));
+      } else if (suffix.equals(CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX)) {
+        rcs.capability = new ResourcePBImpl(
+            ResourceProto.parseFrom(entry.getValue()));
       } else {
         throw new IOException("Unexpected container state key: " + key);
       }
@@ -275,6 +283,20 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException {
+    String key = CONTAINERS_KEY_PREFIX + containerId.toString()
+        + CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX;
+    try {
+      // New value will overwrite old values for the same key
+      db.put(bytes(key),
+          ((ResourcePBImpl) capability).getProto().toByteArray());
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
   public void storeContainerKilled(ContainerId containerId)
       throws IOException {
     String key = CONTAINERS_KEY_PREFIX + containerId.toString()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index ab49543..d5dce9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -88,6 +89,11 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException {
+  }
+
+  @Override
   public void storeContainerKilled(ContainerId containerId)
       throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
index fa66349..e8ccf54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -74,6 +75,7 @@ public abstract class NMStateStoreService extends AbstractService {
     boolean killed = false;
     String diagnostics = "";
     StartContainerRequest startRequest;
+    Resource capability;
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -94,6 +96,10 @@ public abstract class NMStateStoreService extends AbstractService {
     public StartContainerRequest getStartRequest() {
       return startRequest;
     }
+
+    public Resource getCapability() {
+      return capability;
+    }
   }
 
   public static class LocalResourceTrackerState {
@@ -284,6 +290,15 @@ public abstract class NMStateStoreService extends AbstractService {
       throws IOException;
 
   /**
+   * Record that a container resource has been changed
+   * @param containerId the container ID
+   * @param capability the container resource capability
+   * @throws IOException
+   */
+  public abstract void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException;
+
+  /**
    * Record that a container has completed
    * @param containerId the container ID
    * @param exitCode the exit code from the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index 7bde861..4250ac3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -659,7 +659,7 @@ public class TestNodeManagerResync {
         ApplicationACLsManager aclsManager,
         LocalDirsHandlerService dirsHandler) {
       return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
-          metrics, aclsManager, dirsHandler){
+          metrics, dirsHandler){
         @Override
         public void
         setBlockNewContainerRequests(boolean blockNewContainerRequests) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.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/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 4d0aacd..43f1b29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -28,18 +28,30 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -48,9 +60,17 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
@@ -58,6 +78,9 @@ import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -65,6 +88,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
@@ -77,18 +101,50 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestContainerManagerRecovery {
+public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 
-  private NodeManagerMetrics metrics = NodeManagerMetrics.create();
+  public TestContainerManagerRecovery() throws UnsupportedFileSystemException {
+    super();
+  }
+
+  @Override
+  @Before
+  public void setup() throws IOException {
+    localFS.delete(new Path(localDir.getAbsolutePath()), true);
+    localFS.delete(new Path(tmpDir.getAbsolutePath()), true);
+    localFS.delete(new Path(localLogDir.getAbsolutePath()), true);
+    localFS.delete(new Path(remoteLogDir.getAbsolutePath()), true);
+    localDir.mkdir();
+    tmpDir.mkdir();
+    localLogDir.mkdir();
+    remoteLogDir.mkdir();
+    LOG.info("Created localDir in " + localDir.getAbsolutePath());
+    LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath());
+
+    String bindAddress = "0.0.0.0:12345";
+    conf.set(YarnConfiguration.NM_ADDRESS, bindAddress);
+    conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
+    conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    // Default delSrvc
+    delSrvc = createDeletionService();
+    delSrvc.init(conf);
+    exec = createContainerExecutor();
+    dirsHandler = new LocalDirsHandlerService();
+    nodeHealthChecker = new NodeHealthCheckerService(
+        NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
+    nodeHealthChecker.init(conf);
+  }
 
   @Test
   public void testApplicationRecovery() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
-    conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234");
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "yarn_admin_user");
     NMStateStoreService stateStore = new NMMemoryStateStoreService();
@@ -234,6 +290,91 @@ public class TestContainerManagerRecovery {
   }
 
   @Test
+  public void testContainerResizeRecovery() throws Exception {
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
+    NMStateStoreService stateStore = new NMMemoryStateStoreService();
+    stateStore.init(conf);
+    stateStore.start();
+    Context context = createContext(conf, stateStore);
+    ContainerManagerImpl cm = createContainerManager(context, delSrvc);
+    cm.init(conf);
+    cm.start();
+    // add an application by starting a container
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId attemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId cid = ContainerId.newContainerId(attemptId, 1);
+    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, ByteBuffer> serviceData = Collections.emptyMap();
+    Credentials containerCreds = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    containerCreds.writeTokenStorageToStream(dob);
+    ByteBuffer containerTokens = ByteBuffer.wrap(dob.getData(), 0,
+        dob.getLength());
+    Map<ApplicationAccessType, String> acls = Collections.emptyMap();
+    File tmpDir = new File("target",
+        this.getClass().getSimpleName() + "-tmpDir");
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    FileContext localFS = FileContext.getLocalFSFileContext();
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha = RecordFactoryProvider
+        .getRecordFactory(null).newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources = new HashMap<>();
+    localResources.put(destinationFile, rsrc_alpha);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
+        localResources, containerEnv, commands, serviceData,
+        containerTokens, acls);
+    StartContainersResponse startResponse = startContainer(
+        context, cm, cid, clc, null);
+    assertTrue(startResponse.getFailedRequests().isEmpty());
+    assertEquals(1, context.getApplications().size());
+    Application app = context.getApplications().get(appId);
+    assertNotNull(app);
+    // make sure the container reaches RUNNING state
+    waitForNMContainerState(cm, cid,
+        org.apache.hadoop.yarn.server.nodemanager
+            .containermanager.container.ContainerState.RUNNING);
+    Resource targetResource = Resource.newInstance(2048, 2);
+    IncreaseContainersResourceResponse increaseResponse =
+        increaseContainersResource(context, cm, cid, targetResource);
+    assertTrue(increaseResponse.getFailedRequests().isEmpty());
+    // check status
+    ContainerStatus containerStatus = getContainerStatus(context, cm, cid);
+    assertEquals(targetResource, containerStatus.getCapability());
+    // restart and verify container is running and recovered
+    // to the correct size
+    cm.stop();
+    context = createContext(conf, stateStore);
+    cm = createContainerManager(context);
+    cm.init(conf);
+    cm.start();
+    assertEquals(1, context.getApplications().size());
+    app = context.getApplications().get(appId);
+    assertNotNull(app);
+    containerStatus = getContainerStatus(context, cm, cid);
+    assertEquals(targetResource, containerStatus.getCapability());
+  }
+
+  @Test
   public void testContainerCleanupOnShutdown() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId attemptId =
@@ -257,10 +398,8 @@ public class TestContainerManagerRecovery {
         LogAggregationContext.newInstance("includePattern", "excludePattern");
 
     // verify containers are stopped on shutdown without recovery
-    YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, false);
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, false);
-    conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234");
     Context context = createContext(conf, new NMNullStateStoreService());
     ContainerManagerImpl cm = spy(createContainerManager(context));
     cm.init(conf);
@@ -306,12 +445,36 @@ public class TestContainerManagerRecovery {
     verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class));
   }
 
-  private NMContext createContext(YarnConfiguration conf,
+  private ContainerManagerImpl createContainerManager(Context context,
+      DeletionService delSrvc) {
+    return new ContainerManagerImpl(context, exec, delSrvc,
+        mock(NodeStatusUpdater.class), metrics, dirsHandler) {
+      @Override
+      public void
+      setBlockNewContainerRequests(boolean blockNewContainerRequests) {
+        // do nothing
+      }
+      @Override
+      protected void authorizeGetAndStopContainerRequest(
+          ContainerId containerId, Container container,
+          boolean stopRequest, NMTokenIdentifier identifier)
+          throws YarnException {
+        if(container == null || container.getUser().equals("Fail")){
+          throw new YarnException("Reject this container");
+        }
+      }
+    };
+  }
+
+  private NMContext createContext(Configuration conf,
       NMStateStoreService stateStore) {
     NMContext context = new NMContext(new NMContainerTokenSecretManager(
         conf), new NMTokenSecretManagerInNM(), null,
-        new ApplicationACLsManager(conf), stateStore);
-
+        new ApplicationACLsManager(conf), stateStore){
+      public int getHttpPort() {
+        return HTTP_PORT;
+      }
+    };
     // simulate registration with RM
     MasterKey masterKey = new MasterKeyPBImpl();
     masterKey.setKeyId(123);
@@ -349,6 +512,58 @@ public class TestContainerManagerRecovery {
     });
   }
 
+  private IncreaseContainersResourceResponse increaseContainersResource(
+      Context context, final ContainerManagerImpl cm, ContainerId cid,
+      Resource capability) throws Exception {
+    UserGroupInformation user = UserGroupInformation.createRemoteUser(
+        cid.getApplicationAttemptId().toString());
+    // construct container resource increase request
+    final List<Token> increaseTokens = new ArrayList<Token>();
+    // add increase request
+    Token containerToken = TestContainerManager.createContainerToken(
+        cid, 0, context.getNodeId(), user.getShortUserName(),
+        capability, context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    final IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest.newInstance(increaseTokens);
+    NMTokenIdentifier nmToken = new NMTokenIdentifier(
+        cid.getApplicationAttemptId(), context.getNodeId(),
+        user.getShortUserName(),
+        context.getNMTokenSecretManager().getCurrentKey().getKeyId());
+    user.addTokenIdentifier(nmToken);
+    return user.doAs(
+        new PrivilegedExceptionAction<IncreaseContainersResourceResponse>() {
+          @Override
+          public IncreaseContainersResourceResponse run() throws Exception {
+            return cm.increaseContainersResource(increaseRequest);
+          }
+        });
+  }
+
+  private ContainerStatus getContainerStatus(
+      Context context, final ContainerManagerImpl cm, ContainerId cid)
+      throws  Exception {
+    UserGroupInformation user = UserGroupInformation.createRemoteUser(
+        cid.getApplicationAttemptId().toString());
+    NMTokenIdentifier nmToken = new NMTokenIdentifier(
+        cid.getApplicationAttemptId(), context.getNodeId(),
+        user.getShortUserName(),
+        context.getNMTokenSecretManager().getCurrentKey().getKeyId());
+    user.addTokenIdentifier(nmToken);
+    List<ContainerId> containerIds = new ArrayList<>();
+    containerIds.add(cid);
+    final GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    return user.doAs(
+        new PrivilegedExceptionAction<ContainerStatus>() {
+          @Override
+          public ContainerStatus run() throws Exception {
+            return cm.getContainerStatuses(gcsRequest)
+                .getContainerStatuses().get(0);
+          }
+        });
+  }
+
   private void waitForAppState(Application app, ApplicationState state)
       throws Exception {
     final int msecPerSleep = 10;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
index e0487e7..a1c95ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -122,9 +123,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       rcsCopy.killed = rcs.killed;
       rcsCopy.diagnostics = rcs.diagnostics;
       rcsCopy.startRequest = rcs.startRequest;
+      rcsCopy.capability = rcs.capability;
       result.add(rcsCopy);
     }
-    return new ArrayList<RecoveredContainerState>();
+    return result;
   }
 
   @Override
@@ -153,6 +155,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public synchronized void storeContainerResourceChanged(
+      ContainerId containerId, Resource capability) throws IOException {
+    RecoveredContainerState rcs = getRecoveredContainerState(containerId);
+    rcs.capability = capability;
+  }
+
+  @Override
   public synchronized void storeContainerKilled(ContainerId containerId)
       throws IOException {
     RecoveredContainerState rcs = getRecoveredContainerState(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f86eae17/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index 1804424..08b49e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -298,6 +298,17 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(containerReq, rcs.getStartRequest());
     assertEquals(diags.toString(), rcs.getDiagnostics());
 
+    // increase the container size, and verify recovered
+    stateStore.storeContainerResourceChanged(containerId, Resource.newInstance(2468, 4));
+    restartStateStore();
+    recoveredContainers = stateStore.loadContainersState();
+    assertEquals(1, recoveredContainers.size());
+    rcs = recoveredContainers.get(0);
+    assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
+    assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
+    assertEquals(false, rcs.getKilled());
+    assertEquals(Resource.newInstance(2468, 4), rcs.getCapability());
+
     // mark the container killed, add some more diags, and verify recovered
     diags.append("some more diags for container");
     stateStore.storeContainerDiagnostics(containerId, diags);


[10/50] [abbrv] hadoop git commit: HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract.

Posted by wa...@apache.org.
HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract.


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

Branch: refs/heads/YARN-1197
Commit: 7ad3556ed38560585579172aa68356f37b2288c8
Parents: 24f6a7c
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Aug 31 18:17:14 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Aug 31 18:17:14 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java  | 2 +-
 .../src/test/java/org/apache/hadoop/fs/viewfs/ViewFsBaseTest.java | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ad3556e/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 95eb677..0f52d22 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -756,6 +756,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12325. RPC Metrics : Add the ability track and log slow RPCs.
     (Anu Engineer via xyao)
 
+    HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract.
+    (wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ad3556e/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 7fad990..ea4d9b1 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
@@ -77,7 +77,7 @@ import org.junit.Test;
  * </p>
  */
 
-public class ViewFileSystemBaseTest {
+abstract public class ViewFileSystemBaseTest {
   FileSystem fsView;  // the view file system - the mounts are here
   FileSystem fsTarget;  // the target file system - the mount will point here
   Path targetTestRoot;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ad3556e/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 d8ab539..ceebb26 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
@@ -76,7 +76,7 @@ import org.mockito.Mockito;
  *     @AfterClass    public static void ClusterShutdownAtEnd()
  * </p>
  */
-public class ViewFsBaseTest {
+abstract public class ViewFsBaseTest {
   FileContext fcView; // the view file system - the mounts are here
   FileContext fcTarget; // the target file system - the mount will point here
   Path targetTestRoot;


[25/50] [abbrv] hadoop git commit: HDFS-9003. ForkJoin thread pool leaks. Contributed by Kihwal Lee.

Posted by wa...@apache.org.
HDFS-9003. ForkJoin thread pool leaks. 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/de928d56
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/de928d56
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/de928d56

Branch: refs/heads/YARN-1197
Commit: de928d566a119f0b7fa5f171719642cd86be0af7
Parents: afc88b3
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Sep 2 15:19:04 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Sep 2 15:19:04 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                        | 2 ++
 .../main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java  | 1 +
 2 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de928d56/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 78bbf26..7a685f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1279,6 +1279,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8388. Time and Date format need to be in sync in NameNode UI page.
     (Surendra Singh Lilhore via aajisaka)
 
+    HDFS-9003. ForkJoin thread pool leaks. (Kihwal Lee via jing9)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de928d56/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 853c30c..0dd007d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -880,6 +880,7 @@ public class FSImage implements Closeable {
         root, counts);
     p.execute(task);
     task.join();
+    p.shutdown();
     LOG.info("Quota initialization completed in " + (Time.now() - start) +
         " milliseconds\n" + counts);
   }


[18/50] [abbrv] hadoop git commit: HDFS-8995. Flaw in registration bookeeping can make DN die on reconnect. (Kihwal Lee via yliu)

Posted by wa...@apache.org.
HDFS-8995. Flaw in registration bookeeping can make DN die on reconnect. (Kihwal Lee via yliu)


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

Branch: refs/heads/YARN-1197
Commit: 5652131d2ea68c408dd3cd8bee31723642a8cdde
Parents: 4620767
Author: yliu <yl...@apache.org>
Authored: Wed Sep 2 08:58:51 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Wed Sep 2 08:58:51 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                   | 3 +++
 .../apache/hadoop/hdfs/server/datanode/BPOfferService.java    | 3 +--
 .../apache/hadoop/hdfs/server/datanode/BPServiceActor.java    | 7 ++++---
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java | 5 +----
 4 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5652131d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 57ddcb2..ea398f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1302,6 +1302,9 @@ Release 2.7.2 - UNRELEASED
 
     HDFS-8891. HDFS concat should keep srcs order. (Yong Zhang via jing9)
 
+    HDFS-8995. Flaw in registration bookeeping can make DN die on reconnect.
+    (Kihwal Lee via yliu)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5652131d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 92323f1..7aab4f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -351,9 +351,8 @@ class BPOfferService {
             reg.getStorageInfo().getNamespaceID(), "namespace ID");
         checkNSEquality(bpRegistration.getStorageInfo().getClusterID(),
             reg.getStorageInfo().getClusterID(), "cluster ID");
-      } else {
-        bpRegistration = reg;
       }
+      bpRegistration = reg;
 
       dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
       // Add the initial block token secret keys to the DN's secret manager.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5652131d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 1817427..85ea6ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -767,15 +767,16 @@ class BPServiceActor implements Runnable {
   void register(NamespaceInfo nsInfo) throws IOException {
     // The handshake() phase loaded the block pool storage
     // off disk - so update the bpRegistration object from that info
-    bpRegistration = bpos.createRegistration();
+    DatanodeRegistration newBpRegistration = bpos.createRegistration();
 
     LOG.info(this + " beginning handshake with NN");
 
     while (shouldRun()) {
       try {
         // Use returned registration from namenode with updated fields
-        bpRegistration = bpNamenode.registerDatanode(bpRegistration);
-        bpRegistration.setNamespaceInfo(nsInfo);
+        newBpRegistration = bpNamenode.registerDatanode(newBpRegistration);
+        newBpRegistration.setNamespaceInfo(nsInfo);
+        bpRegistration = newBpRegistration;
         break;
       } catch(EOFException e) {  // namenode might have just restarted
         LOG.info("Problem connecting to server: " + nnAddr + " :"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5652131d/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 5bc50b0..42cbd96 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
@@ -1261,10 +1261,7 @@ public class DataNode extends ReconfigurableBase
    */
   synchronized void bpRegistrationSucceeded(DatanodeRegistration bpRegistration,
       String blockPoolId) throws IOException {
-    // Set the ID if we haven't already
-    if (null == id) {
-      id = bpRegistration;
-    }
+    id = bpRegistration;
 
     if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) {
       throw new IOException("Inconsistent Datanode IDs. Name-node returned "


[19/50] [abbrv] hadoop git commit: Revert "MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple mappers. Contributed by Brahma Reddy Battula."

Posted by wa...@apache.org.
Revert "MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple mappers. Contributed by Brahma Reddy Battula."

This reverts commit 75a2560e51387ea31ef4609ef434475bbbc628f7.


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

Branch: refs/heads/YARN-1197
Commit: 00804e24579083d03a67b86c2c57e70b910dda8d
Parents: 5652131
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Sep 2 12:22:16 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Sep 2 12:22:16 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                  |  3 ---
 .../src/test/java/org/apache/hadoop/hdfs/NNBench.java | 14 ++++----------
 2 files changed, 4 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00804e24/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 27af9f9..88cca41 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -509,9 +509,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6389. Fix BaileyBorweinPlouffe CLI usage message. (Brahma Reddy Battula
     via devaraj)
 
-    MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple
-    mappers. (Brahma Reddy Battula via aajisaka)
-
     MAPREDUCE-6373. The logger reports total input paths but it is referring
     to input files. (Bibin A Chundatt via devaraj)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00804e24/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
index 8a1f1ba..b6c0104 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
@@ -57,8 +57,6 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -689,9 +687,6 @@ public class NNBench {
       dataDirName = conf.get("test.nnbench.datadir.name");
       op = conf.get("test.nnbench.operation");
       readFile = conf.getBoolean("test.nnbench.readFileAfterOpen", false);
-      int taskId =
-          TaskAttemptID.forName(conf.get(MRJobConfig.TASK_ATTEMPT_ID))
-              .getTaskID().getId();
       
       long totalTimeTPmS = 0l;
       long startTimeTPmS = 0l;
@@ -704,19 +699,18 @@ public class NNBench {
       successfulFileOps = 0l;
       
       if (barrier()) {
-        String filePrefix = "file_" + taskId + "_";
         if (op.equals(OP_CREATE_WRITE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doCreateWriteOp(filePrefix, reporter);
+          doCreateWriteOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_OPEN_READ)) {
           startTimeTPmS = System.currentTimeMillis();
-          doOpenReadOp(filePrefix, reporter);
+          doOpenReadOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_RENAME)) {
           startTimeTPmS = System.currentTimeMillis();
-          doRenameOp(filePrefix, reporter);
+          doRenameOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_DELETE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doDeleteOp(filePrefix, reporter);
+          doDeleteOp("file_" + hostName + "_", reporter);
         }
         
         endTimeTPms = System.currentTimeMillis();


[41/50] [abbrv] hadoop git commit: YARN-4105. Capacity Scheduler headroom for DRF is wrong. Contributed by Chang Li

Posted by wa...@apache.org.
YARN-4105. Capacity Scheduler headroom for DRF is wrong. Contributed by Chang Li


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

Branch: refs/heads/YARN-1197
Commit: 6eaca2e3634a88dc55689e8960352d6248c424d9
Parents: 40d222e
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Sep 4 15:30:53 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Sep 4 15:30:53 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/capacity/LeafQueue.java           |   2 +-
 .../capacity/TestCapacityScheduler.java         | 112 ++++++++++++++++++-
 3 files changed, 115 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eaca2e3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 29eabb5..662106b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -880,6 +880,9 @@ Release 2.7.2 - UNRELEASED
     YARN-4103. RM WebServices missing scheme for appattempts logLinks.
     (Jonathan Eagles via vvasudeb)
 
+    YARN-4105. Capacity Scheduler headroom for DRF is wrong (Chang Li via
+    jlowe)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eaca2e3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 658eae1..b43f658 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -878,7 +878,7 @@ public class LeafQueue extends AbstractCSQueue {
      *
      */
     Resource headroom = 
-      Resources.min(resourceCalculator, clusterResource,
+      Resources.componentwiseMin(
         Resources.subtract(userLimit, user.getUsed()),
         Resources.subtract(currentResourceLimit, queueUsage.getUsed())
         );

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eaca2e3/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 279299e..44773be 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
@@ -2916,7 +2916,117 @@ public class TestCapacityScheduler {
 
     rm.stop();
   }
-  
+
+  @Test
+  public void testHeadRoomCalculationWithDRC() throws Exception {
+    // test with total cluster resource of 20GB memory and 20 vcores.
+    // the queue where two apps running has user limit 0.8
+    // allocate 10GB memory and 1 vcore to app 1.
+    // app 1 should have headroom
+    // 20GB*0.8 - 10GB = 6GB memory available and 15 vcores.
+    // allocate 1GB memory and 1 vcore to app2.
+    // app 2 should have headroom 20GB - 10 - 1 = 1GB memory,
+    // and 20*0.8 - 1 = 15 vcores.
+
+    CapacitySchedulerConfiguration csconf =
+        new CapacitySchedulerConfiguration();
+    csconf.setResourceComparator(DominantResourceCalculator.class);
+
+    YarnConfiguration conf = new YarnConfiguration(csconf);
+        conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    LeafQueue qb = (LeafQueue)cs.getQueue("default");
+    qb.setUserLimitFactor((float)0.8);
+
+    // add app 1
+    ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
+    ApplicationAttemptId appAttemptId =
+    BuilderUtils.newApplicationAttemptId(appId, 1);
+
+    RMAppAttemptMetrics attemptMetric =
+        new RMAppAttemptMetrics(appAttemptId, rm.getRMContext());
+    RMAppImpl app = mock(RMAppImpl.class);
+    when(app.getApplicationId()).thenReturn(appId);
+    RMAppAttemptImpl attempt = mock(RMAppAttemptImpl.class);
+    when(attempt.getAppAttemptId()).thenReturn(appAttemptId);
+    when(attempt.getRMAppAttemptMetrics()).thenReturn(attemptMetric);
+    when(app.getCurrentAppAttempt()).thenReturn(attempt);
+
+    rm.getRMContext().getRMApps().put(appId, app);
+
+    SchedulerEvent addAppEvent =
+        new AppAddedSchedulerEvent(appId, "default", "user1");
+    cs.handle(addAppEvent);
+    SchedulerEvent addAttemptEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
+    cs.handle(addAttemptEvent);
+
+    // add app 2
+    ApplicationId appId2 = BuilderUtils.newApplicationId(100, 2);
+    ApplicationAttemptId appAttemptId2 =
+    BuilderUtils.newApplicationAttemptId(appId2, 1);
+
+    RMAppAttemptMetrics attemptMetric2 =
+        new RMAppAttemptMetrics(appAttemptId2, rm.getRMContext());
+    RMAppImpl app2 = mock(RMAppImpl.class);
+    when(app2.getApplicationId()).thenReturn(appId2);
+    RMAppAttemptImpl attempt2 = mock(RMAppAttemptImpl.class);
+    when(attempt2.getAppAttemptId()).thenReturn(appAttemptId2);
+    when(attempt2.getRMAppAttemptMetrics()).thenReturn(attemptMetric2);
+    when(app2.getCurrentAppAttempt()).thenReturn(attempt2);
+
+    rm.getRMContext().getRMApps().put(appId2, app2);
+    addAppEvent =
+        new AppAddedSchedulerEvent(appId2, "default", "user2");
+    cs.handle(addAppEvent);
+    addAttemptEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId2, false);
+    cs.handle(addAttemptEvent);
+
+    // add nodes  to cluster, so cluster have 20GB and 20 vcores
+    Resource newResource = Resource.newInstance(10 * GB, 10);
+    RMNode node = MockNodes.newNodeInfo(0, newResource, 1, "127.0.0.1");
+    cs.handle(new NodeAddedSchedulerEvent(node));
+
+    Resource newResource2 = Resource.newInstance(10 * GB, 10);
+    RMNode node2 = MockNodes.newNodeInfo(0, newResource2, 1, "127.0.0.2");
+    cs.handle(new NodeAddedSchedulerEvent(node2));
+
+    FiCaSchedulerApp fiCaApp1 =
+            cs.getSchedulerApplications().get(app.getApplicationId())
+                .getCurrentAppAttempt();
+
+    FiCaSchedulerApp fiCaApp2 =
+            cs.getSchedulerApplications().get(app2.getApplicationId())
+                .getCurrentAppAttempt();
+    Priority u0Priority = TestUtils.createMockPriority(1);
+    RecordFactory recordFactory =
+    RecordFactoryProvider.getRecordFactory(null);
+
+    // allocate container for app1 with 10GB memory and 1 vcore
+    fiCaApp1.updateResourceRequests(Collections.singletonList(
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 10*GB, 1, true,
+            u0Priority, recordFactory)));
+    cs.handle(new NodeUpdateSchedulerEvent(node));
+    cs.handle(new NodeUpdateSchedulerEvent(node2));
+    assertEquals(6*GB, fiCaApp1.getHeadroom().getMemory());
+    assertEquals(15, fiCaApp1.getHeadroom().getVirtualCores());
+
+    // allocate container for app2 with 1GB memory and 1 vcore
+    fiCaApp2.updateResourceRequests(Collections.singletonList(
+        TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true,
+            u0Priority, recordFactory)));
+    cs.handle(new NodeUpdateSchedulerEvent(node));
+    cs.handle(new NodeUpdateSchedulerEvent(node2));
+    assertEquals(9*GB, fiCaApp2.getHeadroom().getMemory());
+    assertEquals(15, fiCaApp2.getHeadroom().getVirtualCores());
+  }
+
   @Test
   public void testDefaultNodeLabelExpressionQueueConfig() throws Exception {
     CapacityScheduler cs = new CapacityScheduler();


[11/50] [abbrv] hadoop git commit: HADOOP-12367. Move TestFileUtil's test resources to resources folder. (wang via yliu)

Posted by wa...@apache.org.
HADOOP-12367. Move TestFileUtil's test resources to resources folder. (wang via yliu)


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

Branch: refs/heads/YARN-1197
Commit: f4d96be6c637ff54903615cff04b365e25bb3229
Parents: 7ad3556
Author: yliu <yl...@apache.org>
Authored: Tue Sep 1 16:20:56 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Tue Sep 1 16:20:56 2015 +0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +++
 hadoop-common-project/hadoop-common/pom.xml     |  19 +------------------
 .../java/org/apache/hadoop/fs/test-untar.tar    | Bin 20480 -> 0 bytes
 .../java/org/apache/hadoop/fs/test-untar.tgz    | Bin 2024 -> 0 bytes
 .../src/test/resources/test-untar.tar           | Bin 0 -> 20480 bytes
 .../src/test/resources/test-untar.tgz           | Bin 0 -> 2024 bytes
 6 files changed, 4 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 0f52d22..14e6fda 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -759,6 +759,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12368. Mark ViewFileSystemBaseTest and ViewFsBaseTest as abstract.
     (wang)
 
+    HADOOP-12367. Move TestFileUtil's test resources to resources folder.
+    (wang via yliu)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 282735d..3ae09a0 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -454,23 +454,6 @@
             </configuration>
           </execution>
           <execution>
-            <id>copy-test-tarballs</id>
-            <phase>process-test-resources</phase>
-            <goals>
-              <goal>run</goal>
-            </goals>
-            <configuration>
-              <target>
-                <copy toDir="${test.cache.data}">
-                  <fileset dir="${basedir}/src/test/java/org/apache/hadoop/fs">
-                    <include name="test-untar.tar"/>
-                    <include name="test-untar.tgz"/>
-                  </fileset>
-                </copy>
-              </target>
-            </configuration>
-          </execution>
-          <execution>
             <phase>pre-site</phase>
             <goals>
               <goal>run</goal>
@@ -505,7 +488,7 @@
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.h</exclude>
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc.c</exclude>
             <exclude>src/main/native/src/org/apache/hadoop/io/compress/lz4/lz4hc_encoder.h</exclude>
-            <exclude>src/test/java/org/apache/hadoop/fs/test-untar.tgz</exclude>
+            <exclude>src/test/resources/test-untar.tgz</exclude>
             <exclude>src/test/resources/test.har/_SUCCESS</exclude>
             <exclude>src/test/resources/test.har/_index</exclude>
             <exclude>src/test/resources/test.har/_masterindex</exclude>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar
deleted file mode 100644
index 949e985..0000000
Binary files a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz
deleted file mode 100644
index 9e9ef40..0000000
Binary files a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/test-untar.tgz and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar
new file mode 100644
index 0000000..949e985
Binary files /dev/null and b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tar differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4d96be6/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz
new file mode 100644
index 0000000..9e9ef40
Binary files /dev/null and b/hadoop-common-project/hadoop-common/src/test/resources/test-untar.tgz differ


[29/50] [abbrv] hadoop git commit: YARN-3970. Add REST api support for Application Priority. Contributed by Naganarasimha G R.

Posted by wa...@apache.org.
YARN-3970. Add REST api support for Application Priority. Contributed by Naganarasimha G R.


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

Branch: refs/heads/YARN-1197
Commit: b469ac531af1bdda01a04ae0b8d39218ca292163
Parents: c92e31b
Author: Varun Vasudev <vv...@apache.org>
Authored: Thu Sep 3 16:35:10 2015 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Thu Sep 3 16:40:10 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   4 +-
 .../scheduler/capacity/CapacityScheduler.java   |  18 +--
 .../webapp/JAXBContextResolver.java             |   2 +-
 .../resourcemanager/webapp/RMWebServices.java   | 116 ++++++++++++++++++
 .../resourcemanager/webapp/dao/AppPriority.java |  44 +++++++
 .../TestRMWebServicesAppsModification.java      | 120 +++++++++++++++++++
 .../src/site/markdown/ResourceManagerRest.md    | 120 +++++++++++++++++++
 8 files changed, 415 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5e0c43d..4e9b411 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -187,6 +187,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3250. Support admin cli interface in for Application Priority.
     (Rohith Sharma K S via jianhe)
 
+    YARN-3970. Add REST api support for Application Priority.
+    (Naganarasimha G R via vvasudev)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index ab29366..9c87eae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -645,7 +645,7 @@ public class ApplicationCLI extends YarnCLI {
     Priority newAppPriority = Priority.newInstance(Integer.parseInt(priority));
     sysout.println("Updating priority of an aplication " + applicationId);
     client.updateApplicationPriority(appId, newAppPriority);
-    sysout.println("Successfully updated the priority of any application "
-        + applicationId);
+    sysout.println("Successfully updated the application with id "
+        + applicationId + " with priority '" + priority + "'");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index b5ccbd9..a7e9d8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -1832,7 +1832,7 @@ public class CapacityScheduler extends
 
       LOG.info("Application '" + applicationId
           + "' is submitted without priority "
-          + "hence considering default queue/cluster priority:"
+          + "hence considering default queue/cluster priority: "
           + priorityFromContext.getPriority());
     }
 
@@ -1846,8 +1846,8 @@ public class CapacityScheduler extends
     appPriority = priorityFromContext;
 
     LOG.info("Priority '" + appPriority.getPriority()
-        + "' is acceptable in queue :" + queueName + "for application:"
-        + applicationId + "for the user: " + user);
+        + "' is acceptable in queue : " + queueName + " for application: "
+        + applicationId + " for the user: " + user);
 
     return appPriority;
   }
@@ -1876,14 +1876,14 @@ public class CapacityScheduler extends
           + "' is not present, hence could not change priority.");
     }
 
-    if (application.getPriority().equals(newPriority)) {
-      return;
-    }
-
     RMApp rmApp = rmContext.getRMApps().get(applicationId);
     appPriority = checkAndGetApplicationPriority(newPriority, rmApp.getUser(),
         rmApp.getQueue(), applicationId);
 
+    if (application.getPriority().equals(appPriority)) {
+      return;
+    }
+
     // Update new priority in Submission Context to keep track in HA
     rmApp.getApplicationSubmissionContext().setPriority(appPriority);
 
@@ -1909,7 +1909,7 @@ public class CapacityScheduler extends
     }
 
     LOG.info("Priority '" + appPriority + "' is updated in queue :"
-        + rmApp.getQueue() + "for application:" + applicationId
-        + "for the user: " + rmApp.getUser());
+        + rmApp.getQueue() + " for application: " + applicationId
+        + " for the user: " + rmApp.getUser());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.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/JAXBContextResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
index 7d7a347..b61072d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/JAXBContextResolver.java
@@ -59,7 +59,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
     final Class[] rootUnwrappedTypes =
         { NewApplication.class, ApplicationSubmissionContextInfo.class,
             ContainerLaunchContextInfo.class, LocalResourceInfo.class,
-            DelegationToken.class, AppQueue.class };
+            DelegationToken.class, AppQueue.class, AppPriority.class };
 
     this.typesContextMap = new HashMap<Class, JAXBContext>();
     context =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index b1fa80a..8e11e2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -122,6 +123,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
@@ -1048,6 +1050,120 @@ public class RMWebServices {
   }
 
   @GET
+  @Path("/apps/{appid}/priority")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public AppPriority getAppPriority(@Context HttpServletRequest hsr,
+      @PathParam("appid") String appId) throws AuthorizationException {
+    init();
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    String userName = "UNKNOWN-USER";
+    if (callerUGI != null) {
+      userName = callerUGI.getUserName();
+    }
+    RMApp app = null;
+    try {
+      app = getRMAppForAppId(appId);
+    } catch (NotFoundException e) {
+      RMAuditLogger.logFailure(userName, AuditConstants.KILL_APP_REQUEST,
+          "UNKNOWN", "RMWebService",
+          "Trying to get state of an absent application " + appId);
+      throw e;
+    }
+
+    AppPriority ret = new AppPriority();
+    ret.setPriority(
+        app.getApplicationSubmissionContext().getPriority().getPriority());
+
+    return ret;
+  }
+
+  @PUT
+  @Path("/apps/{appid}/priority")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      @Context HttpServletRequest hsr, @PathParam("appid") String appId)
+      throws AuthorizationException, YarnException, InterruptedException,
+          IOException {
+    init();
+    if (targetPriority == null) {
+      throw new YarnException("Target Priority cannot be null");
+    }
+
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    if (callerUGI == null) {
+      throw new AuthorizationException(
+          "Unable to obtain user name, user not authenticated");
+    }
+
+    if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
+      return Response.status(Status.FORBIDDEN)
+          .entity("The default static user cannot carry out this operation.")
+          .build();
+    }
+
+    String userName = callerUGI.getUserName();
+    RMApp app = null;
+    try {
+      app = getRMAppForAppId(appId);
+    } catch (NotFoundException e) {
+      RMAuditLogger.logFailure(userName, AuditConstants.KILL_APP_REQUEST,
+          "UNKNOWN", "RMWebService",
+          "Trying to move an absent application " + appId);
+      throw e;
+    }
+    Priority priority = app.getApplicationSubmissionContext().getPriority();
+    if (priority == null
+        || priority.getPriority() != targetPriority.getPriority()) {
+      return modifyApplicationPriority(app, callerUGI,
+          targetPriority.getPriority());
+    }
+    return Response.status(Status.OK).entity(targetPriority).build();
+  }
+
+  private Response modifyApplicationPriority(final RMApp app,
+      UserGroupInformation callerUGI, final int appPriority)
+          throws IOException, InterruptedException {
+    String userName = callerUGI.getUserName();
+    try {
+      callerUGI.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws IOException, YarnException {
+          Priority priority = Priority.newInstance(appPriority);
+          UpdateApplicationPriorityRequest request =
+              UpdateApplicationPriorityRequest
+                  .newInstance(app.getApplicationId(), priority);
+          rm.getClientRMService().updateApplicationPriority(request);
+          return null;
+        }
+      });
+    } catch (UndeclaredThrowableException ue) {
+      // if the root cause is a permissions issue
+      // bubble that up to the user
+      if (ue.getCause() instanceof YarnException) {
+        YarnException ye = (YarnException) ue.getCause();
+        if (ye.getCause() instanceof AccessControlException) {
+          String appId = app.getApplicationId().toString();
+          String msg = "Unauthorized attempt to change priority of appid "
+              + appId + " by remote user " + userName;
+          return Response.status(Status.FORBIDDEN).entity(msg).build();
+        } else if (ye.getMessage().startsWith("Application in")
+            && ye.getMessage().endsWith("state cannot be update priority.")) {
+          return Response.status(Status.BAD_REQUEST).entity(ye.getMessage())
+              .build();
+        } else {
+          throw ue;
+        }
+      } else {
+        throw ue;
+      }
+    }
+    AppPriority ret = new AppPriority(
+        app.getApplicationSubmissionContext().getPriority().getPriority());
+    return Response.status(Status.OK).entity(ret).build();
+  }
+
+  @GET
   @Path("/apps/{appid}/queue")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public AppQueue getAppQueue(@Context HttpServletRequest hsr,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppPriority.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppPriority.java
new file mode 100644
index 0000000..d905635
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppPriority.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.yarn.server.resourcemanager.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "applicationpriority")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AppPriority {
+
+  private int priority;
+
+  public AppPriority() {
+  }
+
+  public AppPriority(int priority) {
+    this.priority = priority;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+  public int getPriority() {
+    return this.priority;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.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/webapp/TestRMWebServicesAppsModification.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
index de4d116..df3c2bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesAppsModification.java
@@ -970,6 +970,92 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
   }
 
   @Test(timeout = 90000)
+  public void testUpdateAppPriority() throws Exception {
+    client().addFilter(new LoggingFilter(System.out));
+
+    if (!(rm.getResourceScheduler() instanceof CapacityScheduler)) {
+      // till the fair scheduler modifications for priority is completed
+      return;
+    }
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    Configuration conf = new Configuration();
+    conf.setInt(YarnConfiguration.MAX_CLUSTER_LEVEL_APPLICATION_PRIORITY, 10);
+    cs.setClusterMaxPriority(conf);
+
+    // default root queue allows anyone to have admin acl
+    CapacitySchedulerConfiguration csconf =
+        new CapacitySchedulerConfiguration();
+    String[] queues = { "default", "test" };
+    csconf.setQueues("root", queues);
+    csconf.setCapacity("root.default", 50.0f);
+    csconf.setCapacity("root.test", 50.0f);
+    csconf.setAcl("root", QueueACL.ADMINISTER_QUEUE, "someuser");
+    csconf.setAcl("root.default", QueueACL.ADMINISTER_QUEUE, "someuser");
+    csconf.setAcl("root.test", QueueACL.ADMINISTER_QUEUE, "someuser");
+    rm.getResourceScheduler().reinitialize(csconf, rm.getRMContext());
+
+    rm.start();
+    MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
+    String[] mediaTypes =
+        { MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML };
+    MediaType[] contentTypes =
+        { MediaType.APPLICATION_JSON_TYPE, MediaType.APPLICATION_XML_TYPE };
+    for (String mediaType : mediaTypes) {
+      for (MediaType contentType : contentTypes) {
+        RMApp app = rm.submitApp(CONTAINER_MB, "", webserviceUserName);
+        amNodeManager.nodeHeartbeat(true);
+        int modifiedPriority = 8;
+        AppPriority priority = new AppPriority(modifiedPriority);
+        Object entity;
+        if (contentType.equals(MediaType.APPLICATION_JSON_TYPE)) {
+          entity = appPriorityToJSON(priority);
+        } else {
+          entity = priority;
+        }
+        ClientResponse response = this
+            .constructWebResource("apps", app.getApplicationId().toString(),
+                "priority")
+            .entity(entity, contentType).accept(mediaType)
+            .put(ClientResponse.class);
+
+        if (!isAuthenticationEnabled()) {
+          assertEquals(Status.UNAUTHORIZED, response.getClientResponseStatus());
+          continue;
+        }
+        assertEquals(Status.OK, response.getClientResponseStatus());
+        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+          verifyAppPriorityJson(response, modifiedPriority);
+        } else {
+          verifyAppPriorityXML(response, modifiedPriority);
+        }
+
+        response = this
+            .constructWebResource("apps", app.getApplicationId().toString(),
+                "priority")
+            .accept(mediaType).get(ClientResponse.class);
+        assertEquals(Status.OK, response.getClientResponseStatus());
+        if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+          verifyAppPriorityJson(response, modifiedPriority);
+        } else {
+          verifyAppPriorityXML(response, modifiedPriority);
+        }
+
+        // check unauthorized
+        app = rm.submitApp(CONTAINER_MB, "", "someuser");
+        amNodeManager.nodeHeartbeat(true);
+        response = this
+            .constructWebResource("apps", app.getApplicationId().toString(),
+                "priority")
+            .entity(entity, contentType).accept(mediaType)
+            .put(ClientResponse.class);
+        assertEquals(Status.FORBIDDEN, response.getClientResponseStatus());
+      }
+    }
+    rm.stop();
+  }
+
+  @Test(timeout = 90000)
   public void testAppMove() throws Exception {
 
     client().addFilter(new LoggingFilter(System.out));
@@ -1049,6 +1135,15 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     rm.stop();
   }
 
+  protected static String appPriorityToJSON(AppPriority targetPriority)
+      throws Exception {
+    StringWriter sw = new StringWriter();
+    JSONJAXBContext ctx = new JSONJAXBContext(AppPriority.class);
+    JSONMarshaller jm = ctx.createJSONMarshaller();
+    jm.marshallToJSON(targetPriority, sw);
+    return sw.toString();
+  }
+
   protected static String appQueueToJSON(AppQueue targetQueue) throws Exception {
     StringWriter sw = new StringWriter();
     JSONJAXBContext ctx = new JSONJAXBContext(AppQueue.class);
@@ -1056,6 +1151,31 @@ public class TestRMWebServicesAppsModification extends JerseyTestBase {
     jm.marshallToJSON(targetQueue, sw);
     return sw.toString();
   }
+ protected static void verifyAppPriorityJson(ClientResponse response,
+      int expectedPriority) throws JSONException {
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    JSONObject json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    int responsePriority = json.getInt("priority");
+    assertEquals(expectedPriority, responsePriority);
+  }
+
+  protected static void verifyAppPriorityXML(ClientResponse response,
+      int expectedPriority)
+          throws ParserConfigurationException, IOException, SAXException {
+    assertEquals(MediaType.APPLICATION_XML_TYPE, response.getType());
+    String xml = response.getEntity(String.class);
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    DocumentBuilder db = dbf.newDocumentBuilder();
+    InputSource is = new InputSource();
+    is.setCharacterStream(new StringReader(xml));
+    Document dom = db.parse(is);
+    NodeList nodes = dom.getElementsByTagName("applicationpriority");
+    assertEquals("incorrect number of elements", 1, nodes.getLength());
+    Element element = (Element) nodes.item(0);
+    int responsePriority = WebServicesTestUtils.getXmlInt(element, "priority");
+    assertEquals(expectedPriority, responsePriority);
+  }
 
   protected static void
       verifyAppQueueJson(ClientResponse response, String queue)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b469ac53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index e16f8c1..f8b8061 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -30,6 +30,7 @@ ResourceManager REST API's.
 * [Cluster Applications API(Submit Application)](#Cluster_Applications_APISubmit_Application)
 * [Cluster Application State API](#Cluster_Application_State_API)
 * [Cluster Application Queue API](#Cluster_Application_Queue_API)
+* [Cluster Application Priority API](#Cluster_Application_Priority_API)
 * [Cluster Delegation Tokens API](#Cluster_Delegation_Tokens_API)
 
 Overview
@@ -2770,6 +2771,125 @@ Response Body:
       <queue>test</queue>
     </appqueue>
 
+Cluster Application Priority API
+-----------------------------
+
+With the application priority API, you can query the priority of a submitted app as well update priority of a running or accepted app using a PUT request specifying the target priority. To perform the PUT operation, authentication has to be setup for the RM web services. In addition, you must be authorized to update the app priority. Currently you can only update the app priority if you're using the Capacity scheduler.
+
+Please note that in order to update priority of an app, you must have an authentication filter setup for the HTTP interface. The functionality requires that a username is set in the HttpServletRequest. If no filter is setup, the response will be an "UNAUTHORIZED" response.
+
+This feature is currently in the alpha stage and may change in the future.
+
+### URI
+
+      * http://<rm http address:port>/ws/v1/cluster/apps/{appid}/priority
+
+### HTTP Operations Supported
+
+      * GET
+      * PUT
+
+### Query Parameters Supported
+
+      None
+
+### Elements of *apppriority* object
+
+When you make a request for the state of an app, the information returned has the following fields
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| priority | int | The application priority |
+
+### Response Examples
+
+**JSON responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/priority
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "priority":0
+    }
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/priority
+
+Request Body:
+
+    {
+      "priority":8
+    }
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/json
+    Transfer-Encoding: chunked
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    {
+      "priority":8
+    }
+
+**XML responses**
+
+HTTP Request
+
+      GET http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/priority
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 98
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <applicationpriority>
+      <priority>0</priority>
+    </applicationpriority>
+
+HTTP Request
+
+      PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/priority
+
+Request Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <applicationpriority>
+      <priority>8</priority>
+    </applicationpriority>
+
+Response Header:
+
+    HTTP/1.1 200 OK
+    Content-Type: application/xml
+    Content-Length: 95
+    Server: Jetty(6.1.26)
+
+Response Body:
+
+    <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+    <applicationpriority>
+      <priority>8</priority>
+    </applicationpriority>
+
 Cluster Delegation Tokens API
 -----------------------------
 


[31/50] [abbrv] hadoop git commit: HADOOP-10318. Incorrect reference to nodeFile in RumenToSLSConverter error message. Contributed by Wei Yan.

Posted by wa...@apache.org.
HADOOP-10318. Incorrect reference to nodeFile in RumenToSLSConverter error message. Contributed by Wei Yan.


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

Branch: refs/heads/YARN-1197
Commit: 355eaaa33d01f06e9efe960b8888fb925e03ffb9
Parents: 9a87f81
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu Sep 3 22:48:53 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Thu Sep 3 22:48:53 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                   | 3 +++
 .../main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/355eaaa3/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e915290..20474be 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1091,6 +1091,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12362. Set hadoop.tmp.dir and hadoop.log.dir in pom.
     (Charlie Helin via wang)
 
+    HADOOP-10318. Incorrect reference to nodeFile in RumenToSLSConverter
+    error message. (Wei Yan via ozawa)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/355eaaa3/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
index 63d022b..0d0745c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java
@@ -110,7 +110,7 @@ public class RumenToSLSConverter {
     if (! nodeFile.getParentFile().exists()
             && ! nodeFile.getParentFile().mkdirs()) {
       System.err.println("ERROR: Cannot create output directory in path: "
-              + jsonFile.getParentFile().getAbsoluteFile());
+              + nodeFile.getParentFile().getAbsoluteFile());
       System.exit(1);
     }
 


[43/50] [abbrv] hadoop git commit: YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
deleted file mode 100644
index 29b0ffe..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.junit.Test;
-
-public class TestContainerResourceDecrease {
-  @Test
-  public void testResourceDecreaseContext() {
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(
-        containerId, resource);
-
-    // get proto and recover to ctx
-    ContainerResourceDecreaseProto proto = 
-        ((ContainerResourceDecreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceDecreasePBImpl(proto);
-
-    // check values
-    Assert.assertEquals(ctx.getCapability(), resource);
-    Assert.assertEquals(ctx.getContainerId(), containerId);
-  }
-  
-  @Test
-  public void testResourceDecreaseContextWithNull() {
-    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(null,
-        null);
-    
-    // get proto and recover to ctx;
-    ContainerResourceDecreaseProto proto = 
-        ((ContainerResourceDecreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceDecreasePBImpl(proto);
-
-    // check values
-    Assert.assertNull(ctx.getCapability());
-    Assert.assertNull(ctx.getContainerId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
deleted file mode 100644
index 932d5a7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api;
-
-import java.util.Arrays;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
-import org.junit.Test;
-
-public class TestContainerResourceIncrease {
-  @Test
-  public void testResourceIncreaseContext() {
-    byte[] identifier = new byte[] { 1, 2, 3, 4 };
-    Token token = Token.newInstance(identifier, "", "".getBytes(), "");
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(
-        containerId, resource, token);
-
-    // get proto and recover to ctx
-    ContainerResourceIncreaseProto proto = 
-        ((ContainerResourceIncreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceIncreasePBImpl(proto);
-
-    // check values
-    Assert.assertEquals(ctx.getCapability(), resource);
-    Assert.assertEquals(ctx.getContainerId(), containerId);
-    Assert.assertTrue(Arrays.equals(ctx.getContainerToken().getIdentifier()
-        .array(), identifier));
-  }
-  
-  @Test
-  public void testResourceIncreaseContextWithNull() {
-    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(null,
-        null, null);
-    
-    // get proto and recover to ctx;
-    ContainerResourceIncreaseProto proto = 
-        ((ContainerResourceIncreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceIncreasePBImpl(proto);
-
-    // check values
-    Assert.assertNull(ctx.getContainerToken());
-    Assert.assertNull(ctx.getCapability());
-    Assert.assertNull(ctx.getContainerId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
deleted file mode 100644
index cf4dabf..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
-import org.junit.Test;
-
-public class TestContainerResourceIncreaseRequest {
-  @Test
-  public void ContainerResourceIncreaseRequest() {
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
-        .newInstance(containerId, resource);
-
-    // to proto and get it back
-    ContainerResourceIncreaseRequestProto proto = 
-        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
-    ContainerResourceIncreaseRequest contextRecover = 
-        new ContainerResourceIncreaseRequestPBImpl(proto);
-
-    // check value
-    Assert.assertEquals(contextRecover.getContainerId(), containerId);
-    Assert.assertEquals(contextRecover.getCapability(), resource);
-  }
-
-  @Test
-  public void testResourceChangeContextWithNullField() {
-    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
-        .newInstance(null, null);
-
-    // to proto and get it back
-    ContainerResourceIncreaseRequestProto proto = 
-        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
-    ContainerResourceIncreaseRequest contextRecover = 
-        new ContainerResourceIncreaseRequestPBImpl(proto);
-
-    // check value
-    Assert.assertNull(contextRecover.getContainerId());
-    Assert.assertNull(contextRecover.getCapability());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 8dacd3b..0d88bf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -113,9 +113,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -155,9 +153,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
@@ -190,9 +186,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
@@ -467,9 +461,7 @@ public class TestPBImplRecords {
     generateByNewInstance(ContainerLaunchContext.class);
     generateByNewInstance(ApplicationSubmissionContext.class);
     generateByNewInstance(ContainerReport.class);
-    generateByNewInstance(ContainerResourceDecrease.class);
-    generateByNewInstance(ContainerResourceIncrease.class);
-    generateByNewInstance(ContainerResourceIncreaseRequest.class);
+    generateByNewInstance(ContainerResourceChangeRequest.class);
     generateByNewInstance(ContainerStatus.class);
     generateByNewInstance(PreemptionContainer.class);
     generateByNewInstance(PreemptionResourceRequest.class);
@@ -955,21 +947,9 @@ public class TestPBImplRecords {
   }
 
   @Test
-  public void testContainerResourceDecreasePBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceDecreasePBImpl.class,
-        ContainerResourceDecreaseProto.class);
-  }
-
-  @Test
-  public void testContainerResourceIncreasePBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceIncreasePBImpl.class,
-        ContainerResourceIncreaseProto.class);
-  }
-
-  @Test
-  public void testContainerResourceIncreaseRequestPBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceIncreaseRequestPBImpl.class,
-        ContainerResourceIncreaseRequestProto.class);
+  public void testContainerResourceChangeRequestPBImpl() throws Exception {
+    validatePBImplRecord(ContainerResourceChangeRequestPBImpl.class,
+        ContainerResourceChangeRequestProto.class);
   }
 
   @Test


[05/50] [abbrv] hadoop git commit: HDFS-8990. Move RemoteBlockReader to hdfs-client module. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
HDFS-8990. Move RemoteBlockReader to hdfs-client module. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: 826ae1c26d31f87d88efc920b271bec7eec2e17a
Parents: caa04de
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Aug 31 13:54:14 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Mon Aug 31 13:54:14 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/RemoteBlockReader.java   | 512 +++++++++++++++++++
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  | 480 +++++++++++++++++
 .../protocol/datatransfer/PacketHeader.java     | 214 ++++++++
 .../protocol/datatransfer/PacketReceiver.java   | 310 +++++++++++
 .../hdfs/util/ByteBufferOutputStream.java       |  49 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   1 -
 .../apache/hadoop/hdfs/RemoteBlockReader.java   | 508 ------------------
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  | 477 -----------------
 .../protocol/datatransfer/PacketHeader.java     | 214 --------
 .../protocol/datatransfer/PacketReceiver.java   | 310 -----------
 .../hdfs/util/ByteBufferOutputStream.java       |  49 --
 .../hdfs/TestClientBlockVerification.java       |   4 +-
 13 files changed, 1570 insertions(+), 1561 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
new file mode 100644
index 0000000..7509da5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
@@ -0,0 +1,512 @@
+/**
+ * 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;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FSInputChecker;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * @deprecated this is an old implementation that is being left around
+ * in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
+ * It will be removed in the next release.
+ */
+@InterfaceAudience.Private
+@Deprecated
+public class RemoteBlockReader extends FSInputChecker implements BlockReader {
+  static final Logger LOG = LoggerFactory.getLogger(FSInputChecker.class);
+
+  private final Peer peer;
+  private final DatanodeID datanodeID;
+  private final DataInputStream in;
+  private DataChecksum checksum;
+
+  /** offset in block of the last chunk received */
+  private long lastChunkOffset = -1;
+  private long lastChunkLen = -1;
+  private long lastSeqNo = -1;
+
+  /** offset in block where reader wants to actually read */
+  private long startOffset;
+
+  private final long blockId;
+
+  /** offset in block of of first chunk - may be less than startOffset
+      if startOffset is not chunk-aligned */
+  private final long firstChunkOffset;
+
+  private final int bytesPerChecksum;
+  private final int checksumSize;
+
+  /**
+   * The total number of bytes we need to transfer from the DN.
+   * This is the amount that the user has requested plus some padding
+   * at the beginning so that the read can begin on a chunk boundary.
+   */
+  private final long bytesNeededToFinish;
+  
+  /**
+   * True if we are reading from a local DataNode.
+   */
+  private final boolean isLocal;
+
+  private boolean eos = false;
+  private boolean sentStatusCode = false;
+  
+  ByteBuffer checksumBytes = null;
+  /** Amount of unread data in the current received packet */
+  int dataLeft = 0;
+  
+  private final PeerCache peerCache;
+  
+  /* FSInputChecker interface */
+  
+  /* same interface as inputStream java.io.InputStream#read()
+   * used by DFSInputStream#read()
+   * This violates one rule when there is a checksum error:
+   * "Read should not modify user buffer before successful read"
+   * because it first reads the data to user buffer and then checks
+   * the checksum.
+   */
+  @Override
+  public synchronized int read(byte[] buf, int off, int len) 
+                               throws IOException {
+    
+    // This has to be set here, *before* the skip, since we can
+    // hit EOS during the skip, in the case that our entire read
+    // is smaller than the checksum chunk.
+    boolean eosBefore = eos;
+
+    //for the first read, skip the extra bytes at the front.
+    if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
+      // Skip these bytes. But don't call this.skip()!
+      int toSkip = (int)(startOffset - firstChunkOffset);
+      if ( super.readAndDiscard(toSkip) != toSkip ) {
+        // should never happen
+        throw new IOException("Could not skip required number of bytes");
+      }
+    }
+    
+    int nRead = super.read(buf, off, len);
+
+    // if eos was set in the previous read, send a status code to the DN
+    if (eos && !eosBefore && nRead >= 0) {
+      if (needChecksum()) {
+        sendReadResult(peer, Status.CHECKSUM_OK);
+      } else {
+        sendReadResult(peer, Status.SUCCESS);
+      }
+    }
+    return nRead;
+  }
+
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    /* How can we make sure we don't throw a ChecksumException, at least
+     * in majority of the cases?. This one throws. */  
+    long nSkipped = 0;
+    while (nSkipped < n) {
+      int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
+      int ret = readAndDiscard(toSkip);
+      if (ret <= 0) {
+        return nSkipped;
+      }
+      nSkipped += ret;
+    }
+    return nSkipped;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new IOException("read() is not expected to be invoked. " +
+                          "Use read(buf, off, len) instead.");
+  }
+  
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    /* Checksum errors are handled outside the BlockReader. 
+     * DFSInputStream does not always call 'seekToNewSource'. In the 
+     * case of pread(), it just tries a different replica without seeking.
+     */ 
+    return false;
+  }
+  
+  @Override
+  public void seek(long pos) throws IOException {
+    throw new IOException("Seek() is not supported in BlockInputChecker");
+  }
+
+  @Override
+  protected long getChunkPosition(long pos) {
+    throw new RuntimeException("getChunkPosition() is not supported, " +
+                               "since seek is not required");
+  }
+  
+  /**
+   * Makes sure that checksumBytes has enough capacity 
+   * and limit is set to the number of checksum bytes needed 
+   * to be read.
+   */
+  private void adjustChecksumBytes(int dataLen) {
+    int requiredSize = 
+      ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
+    if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
+      checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
+    } else {
+      checksumBytes.clear();
+    }
+    checksumBytes.limit(requiredSize);
+  }
+  
+  @Override
+  protected synchronized int readChunk(long pos, byte[] buf, int offset, 
+                                       int len, byte[] checksumBuf) 
+                                       throws IOException {
+    TraceScope scope =
+        Trace.startSpan("RemoteBlockReader#readChunk(" + blockId + ")",
+            Sampler.NEVER);
+    try {
+      return readChunkImpl(pos, buf, offset, len, checksumBuf);
+    } finally {
+      scope.close();
+    }
+  }
+
+  private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
+                                     int len, byte[] checksumBuf)
+                                     throws IOException {
+    // Read one chunk.
+    if (eos) {
+      // Already hit EOF
+      return -1;
+    }
+    
+    // Read one DATA_CHUNK.
+    long chunkOffset = lastChunkOffset;
+    if ( lastChunkLen > 0 ) {
+      chunkOffset += lastChunkLen;
+    }
+    
+    // pos is relative to the start of the first chunk of the read.
+    // chunkOffset is relative to the start of the block.
+    // This makes sure that the read passed from FSInputChecker is the
+    // for the same chunk we expect to be reading from the DN.
+    if ( (pos + firstChunkOffset) != chunkOffset ) {
+      throw new IOException("Mismatch in pos : " + pos + " + " + 
+                            firstChunkOffset + " != " + chunkOffset);
+    }
+
+    // Read next packet if the previous packet has been read completely.
+    if (dataLeft <= 0) {
+      //Read packet headers.
+      PacketHeader header = new PacketHeader();
+      header.readFields(in);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("DFSClient readChunk got header " + header);
+      }
+
+      // Sanity check the lengths
+      if (!header.sanityCheck(lastSeqNo)) {
+           throw new IOException("BlockReader: error in packet header " +
+                                 header);
+      }
+
+      lastSeqNo = header.getSeqno();
+      dataLeft = header.getDataLen();
+      adjustChecksumBytes(header.getDataLen());
+      if (header.getDataLen() > 0) {
+        IOUtils.readFully(in, checksumBytes.array(), 0,
+                          checksumBytes.limit());
+      }
+    }
+
+    // Sanity checks
+    assert len >= bytesPerChecksum;
+    assert checksum != null;
+    assert checksumSize == 0 || (checksumBuf.length % checksumSize == 0);
+
+
+    int checksumsToRead, bytesToRead;
+
+    if (checksumSize > 0) {
+
+      // How many chunks left in our packet - this is a ceiling
+      // since we may have a partial chunk at the end of the file
+      int chunksLeft = (dataLeft - 1) / bytesPerChecksum + 1;
+
+      // How many chunks we can fit in databuffer
+      //  - note this is a floor since we always read full chunks
+      int chunksCanFit = Math.min(len / bytesPerChecksum,
+                                  checksumBuf.length / checksumSize);
+
+      // How many chunks should we read
+      checksumsToRead = Math.min(chunksLeft, chunksCanFit);
+      // How many bytes should we actually read
+      bytesToRead = Math.min(
+        checksumsToRead * bytesPerChecksum, // full chunks
+        dataLeft); // in case we have a partial
+    } else {
+      // no checksum
+      bytesToRead = Math.min(dataLeft, len);
+      checksumsToRead = 0;
+    }
+
+    if ( bytesToRead > 0 ) {
+      // Assert we have enough space
+      assert bytesToRead <= len;
+      assert checksumBytes.remaining() >= checksumSize * checksumsToRead;
+      assert checksumBuf.length >= checksumSize * checksumsToRead;
+      IOUtils.readFully(in, buf, offset, bytesToRead);
+      checksumBytes.get(checksumBuf, 0, checksumSize * checksumsToRead);
+    }
+
+    dataLeft -= bytesToRead;
+    assert dataLeft >= 0;
+
+    lastChunkOffset = chunkOffset;
+    lastChunkLen = bytesToRead;
+
+    // If there's no data left in the current packet after satisfying
+    // this read, and we have satisfied the client read, we expect
+    // an empty packet header from the DN to signify this.
+    // Note that pos + bytesToRead may in fact be greater since the
+    // DN finishes off the entire last chunk.
+    if (dataLeft == 0 &&
+        pos + bytesToRead >= bytesNeededToFinish) {
+
+      // Read header
+      PacketHeader hdr = new PacketHeader();
+      hdr.readFields(in);
+
+      if (!hdr.isLastPacketInBlock() ||
+          hdr.getDataLen() != 0) {
+        throw new IOException("Expected empty end-of-read packet! Header: " +
+                              hdr);
+      }
+
+      eos = true;
+    }
+
+    if ( bytesToRead == 0 ) {
+      return -1;
+    }
+
+    return bytesToRead;
+  }
+  
+  private RemoteBlockReader(String file, String bpid, long blockId,
+      DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
+      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+      DatanodeID datanodeID, PeerCache peerCache) {
+    // Path is used only for printing block and file information in debug
+    super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
+                    ":" + bpid + ":of:"+ file)/*too non path-like?*/,
+          1, verifyChecksum,
+          checksum.getChecksumSize() > 0? checksum : null, 
+          checksum.getBytesPerChecksum(),
+          checksum.getChecksumSize());
+
+    this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
+        createSocketAddr(datanodeID.getXferAddr()));
+    
+    this.peer = peer;
+    this.datanodeID = datanodeID;
+    this.in = in;
+    this.checksum = checksum;
+    this.startOffset = Math.max( startOffset, 0 );
+    this.blockId = blockId;
+
+    // The total number of bytes that we need to transfer from the DN is
+    // the amount that the user wants (bytesToRead), plus the padding at
+    // the beginning in order to chunk-align. Note that the DN may elect
+    // to send more than this amount if the read starts/ends mid-chunk.
+    this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
+
+    this.firstChunkOffset = firstChunkOffset;
+    lastChunkOffset = firstChunkOffset;
+    lastChunkLen = -1;
+
+    bytesPerChecksum = this.checksum.getBytesPerChecksum();
+    checksumSize = this.checksum.getChecksumSize();
+    this.peerCache = peerCache;
+  }
+
+  /**
+   * Create a new BlockReader specifically to satisfy a read.
+   * This method also sends the OP_READ_BLOCK request.
+   *
+   * @param file  File location
+   * @param block  The block object
+   * @param blockToken  The block token for security
+   * @param startOffset  The read offset, relative to block head
+   * @param len  The number of bytes to read
+   * @param bufferSize  The IO buffer size (not the client buffer size)
+   * @param verifyChecksum  Whether to verify checksum
+   * @param clientName  Client name
+   * @return New BlockReader instance, or null on error.
+   */
+  public static RemoteBlockReader newBlockReader(String file,
+                                     ExtendedBlock block, 
+                                     Token<BlockTokenIdentifier> blockToken,
+                                     long startOffset, long len,
+                                     int bufferSize, boolean verifyChecksum,
+                                     String clientName, Peer peer,
+                                     DatanodeID datanodeID,
+                                     PeerCache peerCache,
+                                     CachingStrategy cachingStrategy)
+                                       throws IOException {
+    // in and out will be closed when sock is closed (by the caller)
+    final DataOutputStream out =
+        new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
+    new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
+        verifyChecksum, cachingStrategy);
+    
+    //
+    // Get bytes in block, set streams
+    //
+
+    DataInputStream in = new DataInputStream(
+        new BufferedInputStream(peer.getInputStream(), bufferSize));
+    
+    BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
+        PBHelperClient.vintPrefixed(in));
+    RemoteBlockReader2.checkSuccess(status, peer, block, file);
+    ReadOpChecksumInfoProto checksumInfo =
+      status.getReadOpChecksumInfo();
+    DataChecksum checksum = DataTransferProtoUtil.fromProto(
+        checksumInfo.getChecksum());
+    //Warning when we get CHECKSUM_NULL?
+    
+    // Read the first chunk offset.
+    long firstChunkOffset = checksumInfo.getChunkOffset();
+    
+    if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
+        firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
+      throw new IOException("BlockReader: error in first chunk offset (" +
+                            firstChunkOffset + ") startOffset is " + 
+                            startOffset + " for file " + file);
+    }
+
+    return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
+        in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
+        peer, datanodeID, peerCache);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    startOffset = -1;
+    checksum = null;
+    if (peerCache != null & sentStatusCode) {
+      peerCache.put(datanodeID, peer);
+    } else {
+      peer.close();
+    }
+
+    // in will be closed when its Socket is closed.
+  }
+  
+  @Override
+  public void readFully(byte[] buf, int readOffset, int amtToRead)
+      throws IOException {
+    IOUtils.readFully(this, buf, readOffset, amtToRead);
+  }
+
+  @Override
+  public int readAll(byte[] buf, int offset, int len) throws IOException {
+    return readFully(this, buf, offset, len);
+  }
+
+  /**
+   * When the reader reaches end of the read, it sends a status response
+   * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
+   * closing our connection (which we will re-open), but won't affect
+   * data correctness.
+   */
+  void sendReadResult(Peer peer, Status statusCode) {
+    assert !sentStatusCode : "already sent status code to " + peer;
+    try {
+      RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
+      sentStatusCode = true;
+    } catch (IOException e) {
+      // It's ok not to be able to send this. But something is probably wrong.
+      LOG.info("Could not send read status (" + statusCode + ") to datanode " +
+               peer.getRemoteAddressString() + ": " + e.getMessage());
+    }
+  }
+
+  @Override
+  public int read(ByteBuffer buf) throws IOException {
+    throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
+  }
+  
+  @Override
+  public int available() throws IOException {
+    // An optimistic estimate of how much data is available
+    // to us without doing network I/O.
+    return RemoteBlockReader2.TCP_WINDOW_SIZE;
+  }
+
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+  
+  @Override
+  public boolean isShortCircuit() {
+    return false;
+  }
+
+  @Override
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
new file mode 100644
index 0000000..5541e6d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
@@ -0,0 +1,480 @@
+/**
+ * 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;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.EnumSet;
+import java.util.UUID;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is a wrapper around connection to datanode
+ * and understands checksum, offset etc.
+ *
+ * Terminology:
+ * <dl>
+ * <dt>block</dt>
+ *   <dd>The hdfs block, typically large (~64MB).
+ *   </dd>
+ * <dt>chunk</dt>
+ *   <dd>A block is divided into chunks, each comes with a checksum.
+ *       We want transfers to be chunk-aligned, to be able to
+ *       verify checksums.
+ *   </dd>
+ * <dt>packet</dt>
+ *   <dd>A grouping of chunks used for transport. It contains a
+ *       header, followed by checksum data, followed by real data.
+ *   </dd>
+ * </dl>
+ * Please see DataNode for the RPC specification.
+ *
+ * This is a new implementation introduced in Hadoop 0.23 which
+ * is more efficient and simpler than the older BlockReader
+ * implementation. It should be renamed to RemoteBlockReader
+ * once we are confident in it.
+ */
+@InterfaceAudience.Private
+public class RemoteBlockReader2  implements BlockReader {
+
+  static final Logger LOG = LoggerFactory.getLogger(RemoteBlockReader2.class);
+  static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB;
+
+  final private Peer peer;
+  final private DatanodeID datanodeID;
+  final private PeerCache peerCache;
+  final private long blockId;
+  private final ReadableByteChannel in;
+
+  private DataChecksum checksum;
+  private final PacketReceiver packetReceiver = new PacketReceiver(true);
+
+  private ByteBuffer curDataSlice = null;
+
+  /** offset in block of the last chunk received */
+  private long lastSeqNo = -1;
+
+  /** offset in block where reader wants to actually read */
+  private long startOffset;
+  private final String filename;
+
+  private final int bytesPerChecksum;
+  private final int checksumSize;
+
+  /**
+   * The total number of bytes we need to transfer from the DN.
+   * This is the amount that the user has requested plus some padding
+   * at the beginning so that the read can begin on a chunk boundary.
+   */
+  private long bytesNeededToFinish;
+
+  /**
+   * True if we are reading from a local DataNode.
+   */
+  private final boolean isLocal;
+
+  private final boolean verifyChecksum;
+
+  private boolean sentStatusCode = false;
+
+  @VisibleForTesting
+  public Peer getPeer() {
+    return peer;
+  }
+  
+  @Override
+  public synchronized int read(byte[] buf, int off, int len) 
+                               throws IOException {
+
+    UUID randomId = null;
+    if (LOG.isTraceEnabled()) {
+      randomId = UUID.randomUUID();
+      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
+        randomId.toString(), this.filename,
+        this.datanodeID.getHostName()));
+    }
+
+    if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+      TraceScope scope = Trace.startSpan(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+      try {
+        readNextPacket();
+      } finally {
+        scope.close();
+      }
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("Finishing read #" + randomId));
+    }
+
+    if (curDataSlice.remaining() == 0) {
+      // we're at EOF now
+      return -1;
+    }
+    
+    int nRead = Math.min(curDataSlice.remaining(), len);
+    curDataSlice.get(buf, off, nRead);
+    
+    return nRead;
+  }
+
+
+  @Override
+  public synchronized int read(ByteBuffer buf) throws IOException {
+    if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+      TraceScope scope = Trace.startSpan(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")", Sampler.NEVER);
+      try {
+        readNextPacket();
+      } finally {
+        scope.close();
+      }
+    }
+    if (curDataSlice.remaining() == 0) {
+      // we're at EOF now
+      return -1;
+    }
+
+    int nRead = Math.min(curDataSlice.remaining(), buf.remaining());
+    ByteBuffer writeSlice = curDataSlice.duplicate();
+    writeSlice.limit(writeSlice.position() + nRead);
+    buf.put(writeSlice);
+    curDataSlice.position(writeSlice.position());
+
+    return nRead;
+  }
+
+  private void readNextPacket() throws IOException {
+    //Read packet headers.
+    packetReceiver.receiveNextPacket(in);
+
+    PacketHeader curHeader = packetReceiver.getHeader();
+    curDataSlice = packetReceiver.getDataSlice();
+    assert curDataSlice.capacity() == curHeader.getDataLen();
+    
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("DFSClient readNextPacket got header " + curHeader);
+    }
+
+    // Sanity check the lengths
+    if (!curHeader.sanityCheck(lastSeqNo)) {
+         throw new IOException("BlockReader: error in packet header " +
+                               curHeader);
+    }
+    
+    if (curHeader.getDataLen() > 0) {
+      int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
+      int checksumsLen = chunks * checksumSize;
+
+      assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
+        "checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + 
+          " checksumsLen=" + checksumsLen;
+      
+      lastSeqNo = curHeader.getSeqno();
+      if (verifyChecksum && curDataSlice.remaining() > 0) {
+        // N.B.: the checksum error offset reported here is actually
+        // relative to the start of the block, not the start of the file.
+        // This is slightly misleading, but preserves the behavior from
+        // the older BlockReader.
+        checksum.verifyChunkedSums(curDataSlice,
+            packetReceiver.getChecksumSlice(),
+            filename, curHeader.getOffsetInBlock());
+      }
+      bytesNeededToFinish -= curHeader.getDataLen();
+    }    
+    
+    // First packet will include some data prior to the first byte
+    // the user requested. Skip it.
+    if (curHeader.getOffsetInBlock() < startOffset) {
+      int newPos = (int) (startOffset - curHeader.getOffsetInBlock());
+      curDataSlice.position(newPos);
+    }
+
+    // If we've now satisfied the whole client read, read one last packet
+    // header, which should be empty
+    if (bytesNeededToFinish <= 0) {
+      readTrailingEmptyPacket();
+      if (verifyChecksum) {
+        sendReadResult(Status.CHECKSUM_OK);
+      } else {
+        sendReadResult(Status.SUCCESS);
+      }
+    }
+  }
+  
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    /* How can we make sure we don't throw a ChecksumException, at least
+     * in majority of the cases?. This one throws. */
+    long skipped = 0;
+    while (skipped < n) {
+      long needToSkip = n - skipped;
+      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+        readNextPacket();
+      }
+      if (curDataSlice.remaining() == 0) {
+        // we're at EOF now
+        break;
+      }
+
+      int skip = (int)Math.min(curDataSlice.remaining(), needToSkip);
+      curDataSlice.position(curDataSlice.position() + skip);
+      skipped += skip;
+    }
+    return skipped;
+  }
+
+  private void readTrailingEmptyPacket() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Reading empty packet at end of read");
+    }
+    
+    packetReceiver.receiveNextPacket(in);
+
+    PacketHeader trailer = packetReceiver.getHeader();
+    if (!trailer.isLastPacketInBlock() ||
+       trailer.getDataLen() != 0) {
+      throw new IOException("Expected empty end-of-read packet! Header: " +
+                            trailer);
+    }
+  }
+
+  protected RemoteBlockReader2(String file, String bpid, long blockId,
+      DataChecksum checksum, boolean verifyChecksum,
+      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
+      DatanodeID datanodeID, PeerCache peerCache) {
+    this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
+        createSocketAddr(datanodeID.getXferAddr()));
+    // Path is used only for printing block and file information in debug
+    this.peer = peer;
+    this.datanodeID = datanodeID;
+    this.in = peer.getInputStreamChannel();
+    this.checksum = checksum;
+    this.verifyChecksum = verifyChecksum;
+    this.startOffset = Math.max( startOffset, 0 );
+    this.filename = file;
+    this.peerCache = peerCache;
+    this.blockId = blockId;
+
+    // The total number of bytes that we need to transfer from the DN is
+    // the amount that the user wants (bytesToRead), plus the padding at
+    // the beginning in order to chunk-align. Note that the DN may elect
+    // to send more than this amount if the read starts/ends mid-chunk.
+    this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
+    bytesPerChecksum = this.checksum.getBytesPerChecksum();
+    checksumSize = this.checksum.getChecksumSize();
+  }
+
+
+  @Override
+  public synchronized void close() throws IOException {
+    packetReceiver.close();
+    startOffset = -1;
+    checksum = null;
+    if (peerCache != null && sentStatusCode) {
+      peerCache.put(datanodeID, peer);
+    } else {
+      peer.close();
+    }
+
+    // in will be closed when its Socket is closed.
+  }
+  
+  /**
+   * When the reader reaches end of the read, it sends a status response
+   * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
+   * closing our connection (which we will re-open), but won't affect
+   * data correctness.
+   */
+  void sendReadResult(Status statusCode) {
+    assert !sentStatusCode : "already sent status code to " + peer;
+    try {
+      writeReadResult(peer.getOutputStream(), statusCode);
+      sentStatusCode = true;
+    } catch (IOException e) {
+      // It's ok not to be able to send this. But something is probably wrong.
+      LOG.info("Could not send read status (" + statusCode + ") to datanode " +
+               peer.getRemoteAddressString() + ": " + e.getMessage());
+    }
+  }
+
+  /**
+   * Serialize the actual read result on the wire.
+   */
+  static void writeReadResult(OutputStream out, Status statusCode)
+      throws IOException {
+    
+    ClientReadStatusProto.newBuilder()
+      .setStatus(statusCode)
+      .build()
+      .writeDelimitedTo(out);
+
+    out.flush();
+  }
+  
+  /**
+   * File name to print when accessing a block directly (from servlets)
+   * @param s Address of the block location
+   * @param poolId Block pool ID of the block
+   * @param blockId Block ID of the block
+   * @return string that has a file name for debug purposes
+   */
+  public static String getFileName(final InetSocketAddress s,
+      final String poolId, final long blockId) {
+    return s.toString() + ":" + poolId + ":" + blockId;
+  }
+
+  @Override
+  public int readAll(byte[] buf, int offset, int len) throws IOException {
+    return BlockReaderUtil.readAll(this, buf, offset, len);
+  }
+
+  @Override
+  public void readFully(byte[] buf, int off, int len) throws IOException {
+    BlockReaderUtil.readFully(this, buf, off, len);
+  }
+  
+  /**
+   * Create a new BlockReader specifically to satisfy a read.
+   * This method also sends the OP_READ_BLOCK request.
+   *
+   * @param file  File location
+   * @param block  The block object
+   * @param blockToken  The block token for security
+   * @param startOffset  The read offset, relative to block head
+   * @param len  The number of bytes to read
+   * @param verifyChecksum  Whether to verify checksum
+   * @param clientName  Client name
+   * @param peer  The Peer to use
+   * @param datanodeID  The DatanodeID this peer is connected to
+   * @return New BlockReader instance, or null on error.
+   */
+  public static BlockReader newBlockReader(String file,
+                                     ExtendedBlock block,
+                                     Token<BlockTokenIdentifier> blockToken,
+                                     long startOffset, long len,
+                                     boolean verifyChecksum,
+                                     String clientName,
+                                     Peer peer, DatanodeID datanodeID,
+                                     PeerCache peerCache,
+                                     CachingStrategy cachingStrategy) throws IOException {
+    // in and out will be closed when sock is closed (by the caller)
+    final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
+          peer.getOutputStream()));
+    new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
+        verifyChecksum, cachingStrategy);
+
+    //
+    // Get bytes in block
+    //
+    DataInputStream in = new DataInputStream(peer.getInputStream());
+
+    BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
+        PBHelperClient.vintPrefixed(in));
+    checkSuccess(status, peer, block, file);
+    ReadOpChecksumInfoProto checksumInfo =
+      status.getReadOpChecksumInfo();
+    DataChecksum checksum = DataTransferProtoUtil.fromProto(
+        checksumInfo.getChecksum());
+    //Warning when we get CHECKSUM_NULL?
+
+    // Read the first chunk offset.
+    long firstChunkOffset = checksumInfo.getChunkOffset();
+
+    if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
+        firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
+      throw new IOException("BlockReader: error in first chunk offset (" +
+                            firstChunkOffset + ") startOffset is " +
+                            startOffset + " for file " + file);
+    }
+
+    return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
+        checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
+        datanodeID, peerCache);
+  }
+
+  static void checkSuccess(
+      BlockOpResponseProto status, Peer peer,
+      ExtendedBlock block, String file)
+      throws IOException {
+    String logInfo = "for OP_READ_BLOCK"
+      + ", self=" + peer.getLocalAddressString()
+      + ", remote=" + peer.getRemoteAddressString()
+      + ", for file " + file
+      + ", for pool " + block.getBlockPoolId()
+      + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+    DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
+  }
+  
+  @Override
+  public int available() throws IOException {
+    // An optimistic estimate of how much data is available
+    // to us without doing network I/O.
+    return TCP_WINDOW_SIZE;
+  }
+  
+  @Override
+  public boolean isLocal() {
+    return isLocal;
+  }
+  
+  @Override
+  public boolean isShortCircuit() {
+    return false;
+  }
+
+  @Override
+  public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
new file mode 100644
index 0000000..c9966a7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
@@ -0,0 +1,214 @@
+/**
+ * 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.protocol.datatransfer;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PacketHeaderProto;
+import org.apache.hadoop.hdfs.util.ByteBufferOutputStream;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Shorts;
+import com.google.common.primitives.Ints;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * Header data for each packet that goes through the read/write pipelines.
+ * Includes all of the information about the packet, excluding checksums and
+ * actual data.
+ * 
+ * This data includes:
+ *  - the offset in bytes into the HDFS block of the data in this packet
+ *  - the sequence number of this packet in the pipeline
+ *  - whether or not this is the last packet in the pipeline
+ *  - the length of the data in this packet
+ *  - whether or not this packet should be synced by the DNs.
+ *  
+ * When serialized, this header is written out as a protocol buffer, preceded
+ * by a 4-byte integer representing the full packet length, and a 2-byte short
+ * representing the header length.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class PacketHeader {
+  private static final int MAX_PROTO_SIZE = 
+    PacketHeaderProto.newBuilder()
+      .setOffsetInBlock(0)
+      .setSeqno(0)
+      .setLastPacketInBlock(false)
+      .setDataLen(0)
+      .setSyncBlock(false)
+      .build().getSerializedSize();
+  public static final int PKT_LENGTHS_LEN =
+      Ints.BYTES + Shorts.BYTES;
+  public static final int PKT_MAX_HEADER_LEN =
+      PKT_LENGTHS_LEN + MAX_PROTO_SIZE;
+
+  private int packetLen;
+  private PacketHeaderProto proto;
+
+  public PacketHeader() {
+  }
+
+  public PacketHeader(int packetLen, long offsetInBlock, long seqno,
+                      boolean lastPacketInBlock, int dataLen, boolean syncBlock) {
+    this.packetLen = packetLen;
+    Preconditions.checkArgument(packetLen >= Ints.BYTES,
+        "packet len %s should always be at least 4 bytes",
+        packetLen);
+    
+    PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
+      .setOffsetInBlock(offsetInBlock)
+      .setSeqno(seqno)
+      .setLastPacketInBlock(lastPacketInBlock)
+      .setDataLen(dataLen);
+      
+    if (syncBlock) {
+      // Only set syncBlock if it is specified.
+      // This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
+      // because it changes the length of the packet header, and BlockReceiver
+      // in that version did not support variable-length headers.
+      builder.setSyncBlock(syncBlock);
+    }
+      
+    proto = builder.build();
+  }
+
+  public int getDataLen() {
+    return proto.getDataLen();
+  }
+
+  public boolean isLastPacketInBlock() {
+    return proto.getLastPacketInBlock();
+  }
+
+  public long getSeqno() {
+    return proto.getSeqno();
+  }
+
+  public long getOffsetInBlock() {
+    return proto.getOffsetInBlock();
+  }
+
+  public int getPacketLen() {
+    return packetLen;
+  }
+
+  public boolean getSyncBlock() {
+    return proto.getSyncBlock();
+  }
+
+  @Override
+  public String toString() {
+    return "PacketHeader with packetLen=" + packetLen +
+      " header data: " + 
+      proto.toString();
+  }
+  
+  public void setFieldsFromData(
+      int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
+    this.packetLen = packetLen;
+    proto = PacketHeaderProto.parseFrom(headerData);
+  }
+  
+  public void readFields(ByteBuffer buf) throws IOException {
+    packetLen = buf.getInt();
+    short protoLen = buf.getShort();
+    byte[] data = new byte[protoLen];
+    buf.get(data);
+    proto = PacketHeaderProto.parseFrom(data);
+  }
+  
+  public void readFields(DataInputStream in) throws IOException {
+    this.packetLen = in.readInt();
+    short protoLen = in.readShort();
+    byte[] data = new byte[protoLen];
+    in.readFully(data);
+    proto = PacketHeaderProto.parseFrom(data);
+  }
+
+  /**
+   * @return the number of bytes necessary to write out this header,
+   * including the length-prefixing of the payload and header
+   */
+  public int getSerializedSize() {
+    return PKT_LENGTHS_LEN + proto.getSerializedSize();
+  }
+
+  /**
+   * Write the header into the buffer.
+   * This requires that PKT_HEADER_LEN bytes are available.
+   */
+  public void putInBuffer(final ByteBuffer buf) {
+    assert proto.getSerializedSize() <= MAX_PROTO_SIZE
+      : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
+    try {
+      buf.putInt(packetLen);
+      buf.putShort((short) proto.getSerializedSize());
+      proto.writeTo(new ByteBufferOutputStream(buf));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+  
+  public void write(DataOutputStream out) throws IOException {
+    assert proto.getSerializedSize() <= MAX_PROTO_SIZE
+    : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
+    out.writeInt(packetLen);
+    out.writeShort(proto.getSerializedSize());
+    proto.writeTo(out);
+  }
+  
+  public byte[] getBytes() {
+    ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
+    putInBuffer(buf);
+    return buf.array();
+  }
+
+  /**
+   * Perform a sanity check on the packet, returning true if it is sane.
+   * @param lastSeqNo the previous sequence number received - we expect the current
+   * sequence number to be larger by 1.
+   */
+  public boolean sanityCheck(long lastSeqNo) {
+    // We should only have a non-positive data length for the last packet
+    if (proto.getDataLen() <= 0 && !proto.getLastPacketInBlock()) return false;
+    // The last packet should not contain data
+    if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
+    // Seqnos should always increase by 1 with each packet received
+    if (proto.getSeqno() != lastSeqNo + 1) return false;
+    return true;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof PacketHeader)) return false;
+    PacketHeader other = (PacketHeader)o;
+    return this.proto.equals(other.proto);
+  }
+
+  @Override
+  public int hashCode() {
+    return (int)proto.getSeqno();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
new file mode 100644
index 0000000..c4093b1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
@@ -0,0 +1,310 @@
+/**
+ * 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.protocol.datatransfer;
+
+import java.io.Closeable;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.DirectBufferPool;
+import org.apache.hadoop.io.IOUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to handle reading packets one-at-a-time from the wire.
+ * These packets are used both for reading and writing data to/from
+ * DataNodes.
+ */
+@InterfaceAudience.Private
+public class PacketReceiver implements Closeable {
+
+  /**
+   * The max size of any single packet. This prevents OOMEs when
+   * invalid data is sent.
+   */
+  private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
+
+  static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class);
+  
+  private static final DirectBufferPool bufferPool = new DirectBufferPool();
+  private final boolean useDirectBuffers;
+
+  /**
+   * The entirety of the most recently read packet.
+   * The first PKT_LENGTHS_LEN bytes of this buffer are the
+   * length prefixes.
+   */
+  private ByteBuffer curPacketBuf = null;
+  
+  /**
+   * A slice of {@link #curPacketBuf} which contains just the checksums.
+   */
+  private ByteBuffer curChecksumSlice = null;
+  
+  /**
+   * A slice of {@link #curPacketBuf} which contains just the data.
+   */
+  private ByteBuffer curDataSlice = null;
+
+  /**
+   * The packet header of the most recently read packet.
+   */
+  private PacketHeader curHeader;
+  
+  public PacketReceiver(boolean useDirectBuffers) {
+    this.useDirectBuffers = useDirectBuffers;
+    reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
+  }
+
+  public PacketHeader getHeader() {
+    return curHeader;
+  }
+
+  public ByteBuffer getDataSlice() {
+    return curDataSlice;
+  }
+  
+  public ByteBuffer getChecksumSlice() {
+    return curChecksumSlice;
+  }
+
+  /**
+   * Reads all of the data for the next packet into the appropriate buffers.
+   * 
+   * The data slice and checksum slice members will be set to point to the
+   * user data and corresponding checksums. The header will be parsed and
+   * set.
+   */
+  public void receiveNextPacket(ReadableByteChannel in) throws IOException {
+    doRead(in, null);
+  }
+
+  /**
+   * @see #receiveNextPacket(ReadableByteChannel)
+   */
+  public void receiveNextPacket(InputStream in) throws IOException {
+    doRead(null, in);
+  }
+
+  private void doRead(ReadableByteChannel ch, InputStream in)
+      throws IOException {
+    // Each packet looks like:
+    //   PLEN    HLEN      HEADER     CHECKSUMS  DATA
+    //   32-bit  16-bit   <protobuf>  <variable length>
+    //
+    // PLEN:      Payload length
+    //            = length(PLEN) + length(CHECKSUMS) + length(DATA)
+    //            This length includes its own encoded length in
+    //            the sum for historical reasons.
+    //
+    // HLEN:      Header length
+    //            = length(HEADER)
+    //
+    // HEADER:    the actual packet header fields, encoded in protobuf
+    // CHECKSUMS: the crcs for the data chunk. May be missing if
+    //            checksums were not requested
+    // DATA       the actual block data
+    Preconditions.checkState(curHeader == null || !curHeader.isLastPacketInBlock());
+
+    curPacketBuf.clear();
+    curPacketBuf.limit(PacketHeader.PKT_LENGTHS_LEN);
+    doReadFully(ch, in, curPacketBuf);
+    curPacketBuf.flip();
+    int payloadLen = curPacketBuf.getInt();
+    
+    if (payloadLen < Ints.BYTES) {
+      // The "payload length" includes its own length. Therefore it
+      // should never be less than 4 bytes
+      throw new IOException("Invalid payload length " +
+          payloadLen);
+    }
+    int dataPlusChecksumLen = payloadLen - Ints.BYTES;
+    int headerLen = curPacketBuf.getShort();
+    if (headerLen < 0) {
+      throw new IOException("Invalid header length " + headerLen);
+    }
+    
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
+          " headerLen = " + headerLen);
+    }
+    
+    // Sanity check the buffer size so we don't allocate too much memory
+    // and OOME.
+    int totalLen = payloadLen + headerLen;
+    if (totalLen < 0 || totalLen > MAX_PACKET_SIZE) {
+      throw new IOException("Incorrect value for packet payload size: " +
+                            payloadLen);
+    }
+
+    // Make sure we have space for the whole packet, and
+    // read it.
+    reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN +
+        dataPlusChecksumLen + headerLen);
+    curPacketBuf.clear();
+    curPacketBuf.position(PacketHeader.PKT_LENGTHS_LEN);
+    curPacketBuf.limit(PacketHeader.PKT_LENGTHS_LEN +
+        dataPlusChecksumLen + headerLen);
+    doReadFully(ch, in, curPacketBuf);
+    curPacketBuf.flip();
+    curPacketBuf.position(PacketHeader.PKT_LENGTHS_LEN);
+
+    // Extract the header from the front of the buffer (after the length prefixes)
+    byte[] headerBuf = new byte[headerLen];
+    curPacketBuf.get(headerBuf);
+    if (curHeader == null) {
+      curHeader = new PacketHeader();
+    }
+    curHeader.setFieldsFromData(payloadLen, headerBuf);
+    
+    // Compute the sub-slices of the packet
+    int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
+    if (checksumLen < 0) {
+      throw new IOException("Invalid packet: data length in packet header " + 
+          "exceeds data length received. dataPlusChecksumLen=" +
+          dataPlusChecksumLen + " header: " + curHeader); 
+    }
+    
+    reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
+  }
+  
+  /**
+   * Rewrite the last-read packet on the wire to the given output stream.
+   */
+  public void mirrorPacketTo(DataOutputStream mirrorOut) throws IOException {
+    Preconditions.checkState(!useDirectBuffers,
+        "Currently only supported for non-direct buffers");
+    mirrorOut.write(curPacketBuf.array(),
+        curPacketBuf.arrayOffset(),
+        curPacketBuf.remaining());
+  }
+
+  
+  private static void doReadFully(ReadableByteChannel ch, InputStream in,
+      ByteBuffer buf) throws IOException {
+    if (ch != null) {
+      readChannelFully(ch, buf);
+    } else {
+      Preconditions.checkState(!buf.isDirect(),
+          "Must not use direct buffers with InputStream API");
+      IOUtils.readFully(in, buf.array(),
+          buf.arrayOffset() + buf.position(),
+          buf.remaining());
+      buf.position(buf.position() + buf.remaining());
+    }
+  }
+
+  private void reslicePacket(
+      int headerLen, int checksumsLen, int dataLen) {
+    // Packet structure (refer to doRead() for details):
+    //   PLEN    HLEN      HEADER     CHECKSUMS  DATA
+    //   32-bit  16-bit   <protobuf>  <variable length>
+    //   |--- lenThroughHeader ----|
+    //   |----------- lenThroughChecksums   ----|
+    //   |------------------- lenThroughData    ------| 
+    int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
+    int lenThroughChecksums = lenThroughHeader + checksumsLen;
+    int lenThroughData = lenThroughChecksums + dataLen;
+
+    assert dataLen >= 0 : "invalid datalen: " + dataLen;
+    assert curPacketBuf.position() == lenThroughHeader;
+    assert curPacketBuf.limit() == lenThroughData :
+      "headerLen= " + headerLen + " clen=" + checksumsLen + " dlen=" + dataLen +
+      " rem=" + curPacketBuf.remaining();
+
+    // Slice the checksums.
+    curPacketBuf.position(lenThroughHeader);
+    curPacketBuf.limit(lenThroughChecksums);
+    curChecksumSlice = curPacketBuf.slice();
+
+    // Slice the data.
+    curPacketBuf.position(lenThroughChecksums);
+    curPacketBuf.limit(lenThroughData);
+    curDataSlice = curPacketBuf.slice();
+    
+    // Reset buffer to point to the entirety of the packet (including
+    // length prefixes)
+    curPacketBuf.position(0);
+    curPacketBuf.limit(lenThroughData);
+  }
+
+  
+  private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
+      throws IOException {
+    while (buf.remaining() > 0) {
+      int n = ch.read(buf);
+      if (n < 0) {
+        throw new IOException("Premature EOF reading from " + ch);
+      }
+    }
+  }
+  
+  private void reallocPacketBuf(int atLeastCapacity) {
+    // Realloc the buffer if this packet is longer than the previous
+    // one.
+    if (curPacketBuf == null ||
+        curPacketBuf.capacity() < atLeastCapacity) {
+      ByteBuffer newBuf;
+      if (useDirectBuffers) {
+        newBuf = bufferPool.getBuffer(atLeastCapacity);
+      } else {
+        newBuf = ByteBuffer.allocate(atLeastCapacity);
+      }
+      // If reallocing an existing buffer, copy the old packet length
+      // prefixes over
+      if (curPacketBuf != null) {
+        curPacketBuf.flip();
+        newBuf.put(curPacketBuf);
+      }
+      
+      returnPacketBufToPool();
+      curPacketBuf = newBuf;
+    }
+  }
+  
+  private void returnPacketBufToPool() {
+    if (curPacketBuf != null && curPacketBuf.isDirect()) {
+      bufferPool.returnBuffer(curPacketBuf);
+      curPacketBuf = null;
+    }
+  }
+
+  @Override // Closeable
+  public void close() {
+    returnPacketBufToPool();
+  }
+  
+  @Override
+  protected void finalize() throws Throwable {
+    try {
+      // just in case it didn't get closed, we
+      // may as well still try to return the buffer
+      returnPacketBufToPool();
+    } finally {
+      super.finalize();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
new file mode 100644
index 0000000..31d4dcc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteBufferOutputStream.java
@@ -0,0 +1,49 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * OutputStream that writes into a {@link ByteBuffer}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ByteBufferOutputStream extends OutputStream {
+
+  private final ByteBuffer buf;
+
+  public ByteBufferOutputStream(ByteBuffer buf) {
+    this.buf = buf;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    buf.put((byte)b);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    buf.put(b, off, len);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7b5979e..ef8fac5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -867,6 +867,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8980. Remove unnecessary block replacement in INodeFile. (jing9)
 
+    HDFS-8990. Move RemoteBlockReader to hdfs-client module.
+    (Mingliang via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 3c49ef7..268a5b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -203,7 +203,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     DataEncryptionKeyFactory {
   public static final Log LOG = LogFactory.getLog(DFSClient.class);
   public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
-  static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
 
   private final Configuration conf;
   private final DfsClientConf dfsClientConf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/826ae1c2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
deleted file mode 100644
index 015e154..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ /dev/null
@@ -1,508 +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.hdfs;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FSInputChecker;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-
-/**
- * @deprecated this is an old implementation that is being left around
- * in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
- * It will be removed in the next release.
- */
-@InterfaceAudience.Private
-@Deprecated
-public class RemoteBlockReader extends FSInputChecker implements BlockReader {
-  private final Peer peer;
-  private final DatanodeID datanodeID;
-  private final DataInputStream in;
-  private DataChecksum checksum;
-
-  /** offset in block of the last chunk received */
-  private long lastChunkOffset = -1;
-  private long lastChunkLen = -1;
-  private long lastSeqNo = -1;
-
-  /** offset in block where reader wants to actually read */
-  private long startOffset;
-
-  private final long blockId;
-
-  /** offset in block of of first chunk - may be less than startOffset
-      if startOffset is not chunk-aligned */
-  private final long firstChunkOffset;
-
-  private final int bytesPerChecksum;
-  private final int checksumSize;
-
-  /**
-   * The total number of bytes we need to transfer from the DN.
-   * This is the amount that the user has requested plus some padding
-   * at the beginning so that the read can begin on a chunk boundary.
-   */
-  private final long bytesNeededToFinish;
-  
-  /**
-   * True if we are reading from a local DataNode.
-   */
-  private final boolean isLocal;
-
-  private boolean eos = false;
-  private boolean sentStatusCode = false;
-  
-  ByteBuffer checksumBytes = null;
-  /** Amount of unread data in the current received packet */
-  int dataLeft = 0;
-  
-  private final PeerCache peerCache;
-  
-  /* FSInputChecker interface */
-  
-  /* same interface as inputStream java.io.InputStream#read()
-   * used by DFSInputStream#read()
-   * This violates one rule when there is a checksum error:
-   * "Read should not modify user buffer before successful read"
-   * because it first reads the data to user buffer and then checks
-   * the checksum.
-   */
-  @Override
-  public synchronized int read(byte[] buf, int off, int len) 
-                               throws IOException {
-    
-    // This has to be set here, *before* the skip, since we can
-    // hit EOS during the skip, in the case that our entire read
-    // is smaller than the checksum chunk.
-    boolean eosBefore = eos;
-
-    //for the first read, skip the extra bytes at the front.
-    if (lastChunkLen < 0 && startOffset > firstChunkOffset && len > 0) {
-      // Skip these bytes. But don't call this.skip()!
-      int toSkip = (int)(startOffset - firstChunkOffset);
-      if ( super.readAndDiscard(toSkip) != toSkip ) {
-        // should never happen
-        throw new IOException("Could not skip required number of bytes");
-      }
-    }
-    
-    int nRead = super.read(buf, off, len);
-
-    // if eos was set in the previous read, send a status code to the DN
-    if (eos && !eosBefore && nRead >= 0) {
-      if (needChecksum()) {
-        sendReadResult(peer, Status.CHECKSUM_OK);
-      } else {
-        sendReadResult(peer, Status.SUCCESS);
-      }
-    }
-    return nRead;
-  }
-
-  @Override
-  public synchronized long skip(long n) throws IOException {
-    /* How can we make sure we don't throw a ChecksumException, at least
-     * in majority of the cases?. This one throws. */  
-    long nSkipped = 0;
-    while (nSkipped < n) {
-      int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
-      int ret = readAndDiscard(toSkip);
-      if (ret <= 0) {
-        return nSkipped;
-      }
-      nSkipped += ret;
-    }
-    return nSkipped;
-  }
-
-  @Override
-  public int read() throws IOException {
-    throw new IOException("read() is not expected to be invoked. " +
-                          "Use read(buf, off, len) instead.");
-  }
-  
-  @Override
-  public boolean seekToNewSource(long targetPos) throws IOException {
-    /* Checksum errors are handled outside the BlockReader. 
-     * DFSInputStream does not always call 'seekToNewSource'. In the 
-     * case of pread(), it just tries a different replica without seeking.
-     */ 
-    return false;
-  }
-  
-  @Override
-  public void seek(long pos) throws IOException {
-    throw new IOException("Seek() is not supported in BlockInputChecker");
-  }
-
-  @Override
-  protected long getChunkPosition(long pos) {
-    throw new RuntimeException("getChunkPosition() is not supported, " +
-                               "since seek is not required");
-  }
-  
-  /**
-   * Makes sure that checksumBytes has enough capacity 
-   * and limit is set to the number of checksum bytes needed 
-   * to be read.
-   */
-  private void adjustChecksumBytes(int dataLen) {
-    int requiredSize = 
-      ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
-    if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
-      checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
-    } else {
-      checksumBytes.clear();
-    }
-    checksumBytes.limit(requiredSize);
-  }
-  
-  @Override
-  protected synchronized int readChunk(long pos, byte[] buf, int offset, 
-                                       int len, byte[] checksumBuf) 
-                                       throws IOException {
-    TraceScope scope =
-        Trace.startSpan("RemoteBlockReader#readChunk(" + blockId + ")",
-            Sampler.NEVER);
-    try {
-      return readChunkImpl(pos, buf, offset, len, checksumBuf);
-    } finally {
-      scope.close();
-    }
-  }
-
-  private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
-                                     int len, byte[] checksumBuf)
-                                     throws IOException {
-    // Read one chunk.
-    if (eos) {
-      // Already hit EOF
-      return -1;
-    }
-    
-    // Read one DATA_CHUNK.
-    long chunkOffset = lastChunkOffset;
-    if ( lastChunkLen > 0 ) {
-      chunkOffset += lastChunkLen;
-    }
-    
-    // pos is relative to the start of the first chunk of the read.
-    // chunkOffset is relative to the start of the block.
-    // This makes sure that the read passed from FSInputChecker is the
-    // for the same chunk we expect to be reading from the DN.
-    if ( (pos + firstChunkOffset) != chunkOffset ) {
-      throw new IOException("Mismatch in pos : " + pos + " + " + 
-                            firstChunkOffset + " != " + chunkOffset);
-    }
-
-    // Read next packet if the previous packet has been read completely.
-    if (dataLeft <= 0) {
-      //Read packet headers.
-      PacketHeader header = new PacketHeader();
-      header.readFields(in);
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("DFSClient readChunk got header " + header);
-      }
-
-      // Sanity check the lengths
-      if (!header.sanityCheck(lastSeqNo)) {
-           throw new IOException("BlockReader: error in packet header " +
-                                 header);
-      }
-
-      lastSeqNo = header.getSeqno();
-      dataLeft = header.getDataLen();
-      adjustChecksumBytes(header.getDataLen());
-      if (header.getDataLen() > 0) {
-        IOUtils.readFully(in, checksumBytes.array(), 0,
-                          checksumBytes.limit());
-      }
-    }
-
-    // Sanity checks
-    assert len >= bytesPerChecksum;
-    assert checksum != null;
-    assert checksumSize == 0 || (checksumBuf.length % checksumSize == 0);
-
-
-    int checksumsToRead, bytesToRead;
-
-    if (checksumSize > 0) {
-
-      // How many chunks left in our packet - this is a ceiling
-      // since we may have a partial chunk at the end of the file
-      int chunksLeft = (dataLeft - 1) / bytesPerChecksum + 1;
-
-      // How many chunks we can fit in databuffer
-      //  - note this is a floor since we always read full chunks
-      int chunksCanFit = Math.min(len / bytesPerChecksum,
-                                  checksumBuf.length / checksumSize);
-
-      // How many chunks should we read
-      checksumsToRead = Math.min(chunksLeft, chunksCanFit);
-      // How many bytes should we actually read
-      bytesToRead = Math.min(
-        checksumsToRead * bytesPerChecksum, // full chunks
-        dataLeft); // in case we have a partial
-    } else {
-      // no checksum
-      bytesToRead = Math.min(dataLeft, len);
-      checksumsToRead = 0;
-    }
-
-    if ( bytesToRead > 0 ) {
-      // Assert we have enough space
-      assert bytesToRead <= len;
-      assert checksumBytes.remaining() >= checksumSize * checksumsToRead;
-      assert checksumBuf.length >= checksumSize * checksumsToRead;
-      IOUtils.readFully(in, buf, offset, bytesToRead);
-      checksumBytes.get(checksumBuf, 0, checksumSize * checksumsToRead);
-    }
-
-    dataLeft -= bytesToRead;
-    assert dataLeft >= 0;
-
-    lastChunkOffset = chunkOffset;
-    lastChunkLen = bytesToRead;
-
-    // If there's no data left in the current packet after satisfying
-    // this read, and we have satisfied the client read, we expect
-    // an empty packet header from the DN to signify this.
-    // Note that pos + bytesToRead may in fact be greater since the
-    // DN finishes off the entire last chunk.
-    if (dataLeft == 0 &&
-        pos + bytesToRead >= bytesNeededToFinish) {
-
-      // Read header
-      PacketHeader hdr = new PacketHeader();
-      hdr.readFields(in);
-
-      if (!hdr.isLastPacketInBlock() ||
-          hdr.getDataLen() != 0) {
-        throw new IOException("Expected empty end-of-read packet! Header: " +
-                              hdr);
-      }
-
-      eos = true;
-    }
-
-    if ( bytesToRead == 0 ) {
-      return -1;
-    }
-
-    return bytesToRead;
-  }
-  
-  private RemoteBlockReader(String file, String bpid, long blockId,
-      DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
-      long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
-      DatanodeID datanodeID, PeerCache peerCache) {
-    // Path is used only for printing block and file information in debug
-    super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
-                    ":" + bpid + ":of:"+ file)/*too non path-like?*/,
-          1, verifyChecksum,
-          checksum.getChecksumSize() > 0? checksum : null, 
-          checksum.getBytesPerChecksum(),
-          checksum.getChecksumSize());
-
-    this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
-        createSocketAddr(datanodeID.getXferAddr()));
-    
-    this.peer = peer;
-    this.datanodeID = datanodeID;
-    this.in = in;
-    this.checksum = checksum;
-    this.startOffset = Math.max( startOffset, 0 );
-    this.blockId = blockId;
-
-    // The total number of bytes that we need to transfer from the DN is
-    // the amount that the user wants (bytesToRead), plus the padding at
-    // the beginning in order to chunk-align. Note that the DN may elect
-    // to send more than this amount if the read starts/ends mid-chunk.
-    this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
-
-    this.firstChunkOffset = firstChunkOffset;
-    lastChunkOffset = firstChunkOffset;
-    lastChunkLen = -1;
-
-    bytesPerChecksum = this.checksum.getBytesPerChecksum();
-    checksumSize = this.checksum.getChecksumSize();
-    this.peerCache = peerCache;
-  }
-
-  /**
-   * Create a new BlockReader specifically to satisfy a read.
-   * This method also sends the OP_READ_BLOCK request.
-   *
-   * @param file  File location
-   * @param block  The block object
-   * @param blockToken  The block token for security
-   * @param startOffset  The read offset, relative to block head
-   * @param len  The number of bytes to read
-   * @param bufferSize  The IO buffer size (not the client buffer size)
-   * @param verifyChecksum  Whether to verify checksum
-   * @param clientName  Client name
-   * @return New BlockReader instance, or null on error.
-   */
-  public static RemoteBlockReader newBlockReader(String file,
-                                     ExtendedBlock block, 
-                                     Token<BlockTokenIdentifier> blockToken,
-                                     long startOffset, long len,
-                                     int bufferSize, boolean verifyChecksum,
-                                     String clientName, Peer peer,
-                                     DatanodeID datanodeID,
-                                     PeerCache peerCache,
-                                     CachingStrategy cachingStrategy)
-                                       throws IOException {
-    // in and out will be closed when sock is closed (by the caller)
-    final DataOutputStream out =
-        new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
-    new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
-        verifyChecksum, cachingStrategy);
-    
-    //
-    // Get bytes in block, set streams
-    //
-
-    DataInputStream in = new DataInputStream(
-        new BufferedInputStream(peer.getInputStream(), bufferSize));
-    
-    BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
-        PBHelperClient.vintPrefixed(in));
-    RemoteBlockReader2.checkSuccess(status, peer, block, file);
-    ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
-    DataChecksum checksum = DataTransferProtoUtil.fromProto(
-        checksumInfo.getChecksum());
-    //Warning when we get CHECKSUM_NULL?
-    
-    // Read the first chunk offset.
-    long firstChunkOffset = checksumInfo.getChunkOffset();
-    
-    if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
-        firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
-      throw new IOException("BlockReader: error in first chunk offset (" +
-                            firstChunkOffset + ") startOffset is " + 
-                            startOffset + " for file " + file);
-    }
-
-    return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
-        in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
-        peer, datanodeID, peerCache);
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    startOffset = -1;
-    checksum = null;
-    if (peerCache != null & sentStatusCode) {
-      peerCache.put(datanodeID, peer);
-    } else {
-      peer.close();
-    }
-
-    // in will be closed when its Socket is closed.
-  }
-  
-  @Override
-  public void readFully(byte[] buf, int readOffset, int amtToRead)
-      throws IOException {
-    IOUtils.readFully(this, buf, readOffset, amtToRead);
-  }
-
-  @Override
-  public int readAll(byte[] buf, int offset, int len) throws IOException {
-    return readFully(this, buf, offset, len);
-  }
-
-  /**
-   * When the reader reaches end of the read, it sends a status response
-   * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
-   * closing our connection (which we will re-open), but won't affect
-   * data correctness.
-   */
-  void sendReadResult(Peer peer, Status statusCode) {
-    assert !sentStatusCode : "already sent status code to " + peer;
-    try {
-      RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
-      sentStatusCode = true;
-    } catch (IOException e) {
-      // It's ok not to be able to send this. But something is probably wrong.
-      LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               peer.getRemoteAddressString() + ": " + e.getMessage());
-    }
-  }
-
-  @Override
-  public int read(ByteBuffer buf) throws IOException {
-    throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
-  }
-  
-  @Override
-  public int available() throws IOException {
-    // An optimistic estimate of how much data is available
-    // to us without doing network I/O.
-    return DFSClient.TCP_WINDOW_SIZE;
-  }
-
-  @Override
-  public boolean isLocal() {
-    return isLocal;
-  }
-  
-  @Override
-  public boolean isShortCircuit() {
-    return false;
-  }
-
-  @Override
-  public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
-    return null;
-  }
-}


[33/50] [abbrv] hadoop git commit: HDFS-9009. Send metrics logs to NullAppender by default. (Arpit Agarwal)

Posted by wa...@apache.org.
HDFS-9009. Send metrics logs to NullAppender by default. (Arpit Agarwal)


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

Branch: refs/heads/YARN-1197
Commit: 524ba8708b8e3e17e806748e1f819dec2183bf94
Parents: 0ebc658
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 3 11:14:47 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 3 11:14:47 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/log4j.properties                  | 3 ++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 2 ++
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/524ba870/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
index 1012e20..c26fed4 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
+++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
@@ -152,7 +152,8 @@ log4j.appender.RFAAUDIT.MaxBackupIndex=${hdfs.audit.log.maxbackupindex}
 # NameNode metrics logging.
 # The default is to retain two namenode-metrics.log files up to 64MB each.
 #
-log4j.logger.NameNodeMetricsLog=INFO,NNMETRICSRFA
+namenode.metrics.logger=INFO,NullAppender
+log4j.logger.NameNodeMetricsLog=${namenode.metrics.logger}
 log4j.additivity.NameNodeMetricsLog=false
 log4j.appender.NNMETRICSRFA=org.apache.log4j.RollingFileAppender
 log4j.appender.NNMETRICSRFA.File=${hadoop.log.dir}/namenode-metrics.log

http://git-wip-us.apache.org/repos/asf/hadoop/blob/524ba870/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 275beb2..275dce2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1287,6 +1287,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8885. ByteRangeInputStream used in webhdfs does not override
     available(). (Shradha Revankar via aajisaka)
 
+    HDFS-9009. Send metrics logs to NullAppender by default. (Arpit Agarwal)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[20/50] [abbrv] hadoop git commit: HDFS-8388. Time and Date format need to be in sync in NameNode UI page. Contributed by Surendra Singh Lilhore.

Posted by wa...@apache.org.
HDFS-8388. Time and Date format need to be in sync in NameNode UI page. Contributed by Surendra Singh Lilhore.


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

Branch: refs/heads/YARN-1197
Commit: 65ccf2b1252a5c83755fa24a93cf1d30ee59b2c3
Parents: 00804e2
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Sep 2 14:28:38 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Sep 2 14:28:38 2015 +0900

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md               | 2 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                  | 3 +++
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 5 +++++
 .../apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java   | 6 ++++++
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html         | 5 ++---
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js           | 6 +++---
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.html          | 1 +
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.js            | 2 +-
 .../hadoop-hdfs/src/main/webapps/static/dfs-dust.js          | 8 +++++++-
 9 files changed, 30 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index de706ad..8722968 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -192,6 +192,8 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `PutImageNumOps` | Total number of fsimage uploads to SecondaryNameNode |
 | `PutImageAvgTime` | Average fsimage upload time in milliseconds |
 | `TotalFileOps`| Total number of file operations performed |
+| `NNStarted`| NameNode start time |
+| `NNStartedTimeInMillis`| NameNode start time in milliseconds |
 
 FSNamesystem
 ------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ea398f2..14a9248 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1270,6 +1270,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8950. NameNode refresh doesn't remove DataNodes that are no longer in
     the allowed list (Daniel Templeton)
 
+    HDFS-8388. Time and Date format need to be in sync in NameNode UI page.
+    (Surendra Singh Lilhore via aajisaka)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f4952f7..adcb1d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -6131,6 +6131,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return getStartTime().toString();
   }
 
+  @Override // NameNodeMXBean
+  public long getNNStartedTimeInMillis() {
+    return startTime;
+  }
+
   @Override  // NameNodeMXBean
   public String getCompileInfo() {
     return VersionInfo.getDate() + " by " + VersionInfo.getUser() +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index 0e4d445..00c1abe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -239,6 +239,12 @@ public interface NameNodeMXBean {
   public String getNNStarted();
 
   /**
+   * Gets the NN start time in milliseconds.
+   * @return the NN start time in msec
+   */
+  long getNNStartedTimeInMillis();
+
+  /**
    * Get the compilation information which contains date, user and branch
    *
    * @return the compilation information, as a JSON string.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
index 38808ca..36f8bfe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
@@ -130,9 +130,9 @@
   <tr><th>Namenode ID:</th><td>{NamenodeID}</td></tr>
 {/HAInfo}
 {#nn}
-  <tr><th>Started:</th><td>{NNStarted}</td></tr>
+  <tr><th>Started:</th><td>{NNStartedTimeInMillis|date_tostring}</td></tr>
   <tr><th>Version:</th><td>{Version}</td></tr>
-  <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
+  <tr><th>Compiled:</th><td>{CompileInfo|format_compile_info}</td></tr>
   <tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
   <tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
 {/nn}
@@ -423,7 +423,6 @@ There are no reported volume failures.
 </script><script type="text/javascript" src="/static/bootstrap-3.0.2/js/bootstrap.min.js">
 </script><script type="text/javascript" src="/static/dataTables.bootstrap.js">
 </script><script type="text/javascript" src="/static/moment.min.js">
-</script><script type="text/javascript" src="/static/moment.min.js">
 </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
 </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
 </script><script type="text/javascript" src="/static/dfs-dust.js">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
index 03b27fe..9bc1b5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
@@ -57,7 +57,7 @@
 
       'helper_date_tostring' : function (chunk, ctx, bodies, params) {
         var value = dust.helpers.tap(params.value, chunk, ctx);
-        return chunk.write('' + new Date(Number(value)).toLocaleString());
+        return chunk.write('' + moment(Number(value)).format('ddd MMM DD HH:mm:ss ZZ YYYY'));
       }
     };
 
@@ -175,7 +175,7 @@
     var HELPERS = {
       'helper_relative_time' : function (chunk, ctx, bodies, params) {
         var value = dust.helpers.tap(params.value, chunk, ctx);
-        return chunk.write(moment().subtract(Number(value), 'seconds').format('YYYY-MM-DD HH:mm:ss'));
+        return chunk.write(moment().subtract(Number(value), 'seconds').format('ddd MMM DD HH:mm:ss ZZ YYYY'));
       },
       'helper_usage_bar' : function (chunk, ctx, bodies, params) {
         var value = dust.helpers.tap(params.value, chunk, ctx);
@@ -262,7 +262,7 @@
     var HELPERS = {
       'helper_date_tostring' : function (chunk, ctx, bodies, params) {
         var value = dust.helpers.tap(params.value, chunk, ctx);
-        return chunk.write('' + new Date(Number(value)).toLocaleString());
+        return chunk.write('' + moment(Number(value)).format('ddd MMM DD HH:mm:ss ZZ YYYY'));
       }
     };
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index fbea6ad..e805915 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -199,6 +199,7 @@
     </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
     </script><script type="text/javascript" src="/static/dfs-dust.js">
     </script><script type="text/javascript" src="explorer.js">
+    </script><script type="text/javascript" src="/static/moment.min.js">
     </script>
   </body>
 </html>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 5572880..46f48b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -146,7 +146,7 @@
     var HELPERS = {
       'helper_date_tostring' : function (chunk, ctx, bodies, params) {
         var value = dust.helpers.tap(params.value, chunk, ctx);
-        return chunk.write('' + new Date(Number(value)).toLocaleString());
+        return chunk.write('' + moment(Number(value)).format('ddd MMM DD HH:mm:ss ZZ YYYY'));
       }
     };
     var url = '/webhdfs/v1' + encode_path(dir) + '?op=LISTSTATUS';

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65ccf2b1/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
index 3c8efd9..466e058 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
@@ -58,9 +58,15 @@
     },
 
     'date_tostring' : function (v) {
-      return new Date(Number(v)).toLocaleString();
+      return moment(Number(v)).format('ddd MMM DD HH:mm:ss ZZ YYYY');
     },
 
+    'format_compile_info' : function (v) {
+      var info = v.split(" by ")
+      var date = moment(info[0]).format('ddd MMM DD HH:mm:ss ZZ YYYY');
+      return date.concat(" by ").concat(info[1]);
+     },
+
     'helper_to_permission': function (v) {
       var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
       var vInt = parseInt(v, 8);


[17/50] [abbrv] hadoop git commit: YARN-3222. Added the missing CHANGES.txt entry.

Posted by wa...@apache.org.
YARN-3222. Added the missing CHANGES.txt entry.


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

Branch: refs/heads/YARN-1197
Commit: 4620767156ecc43424bc6c7c4d50519e2563cc69
Parents: 0eb9b19
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue Sep 1 13:43:10 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Sep 1 13:43:10 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/46207671/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e2b1307..b33111c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1701,6 +1701,9 @@ Release 2.7.0 - 2015-04-20
     YARN-3265. Fixed a deadlock in CapacityScheduler by always passing a queue's
     available resource-limit from the parent queue. (Wangda Tan via vinodkv)
 
+    YARN-3222. Fixed RMNode to send scheduler events in sequential order when a
+    node reconnects. (Rohith Sharma K S via jianhe)
+
     YARN-3131. YarnClientImpl should check FAILED and KILLED state in
     submitApplication (Chang Li via jlowe)
     


[03/50] [abbrv] hadoop git commit: HDFS-8980. Remove unnecessary block replacement in INodeFile. Contributed by Jing Zhao.

Posted by wa...@apache.org.
HDFS-8980. Remove unnecessary block replacement in INodeFile. Contributed by Jing Zhao.


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

Branch: refs/heads/YARN-1197
Commit: caa04de149030691b7bc952b534c6128db217ed2
Parents: cf83156
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Aug 31 11:48:09 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Mon Aug 31 11:48:09 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hdfs/server/blockmanagement/BlockInfo.java  | 19 +------
 .../blockmanagement/BlockInfoContiguous.java    | 15 -----
 .../server/blockmanagement/BlockManager.java    | 58 +++++++-------------
 .../hdfs/server/blockmanagement/BlocksMap.java  | 16 ------
 .../hdfs/server/namenode/FSEditLogLoader.java   |  8 +--
 6 files changed, 29 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 3382f81..7b5979e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -865,6 +865,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8983. NameNode support for protected directories. (Arpit Agarwal)
 
+    HDFS-8980. Remove unnecessary block replacement in INodeFile. (jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 706cbcd..810784d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -36,7 +36,7 @@ import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
  * the block are stored.
  */
 @InterfaceAudience.Private
-public abstract class  BlockInfo extends Block
+public abstract class BlockInfo extends Block
     implements LightWeightGSet.LinkedElement {
 
   public static final BlockInfo[] EMPTY_ARRAY = {};
@@ -207,12 +207,6 @@ public abstract class  BlockInfo extends Block
   abstract boolean removeStorage(DatanodeStorageInfo storage);
 
   /**
-   * Replace the current BlockInfo with the new one in corresponding
-   * DatanodeStorageInfo's linked list
-   */
-  abstract void replaceBlock(BlockInfo newBlock);
-
-  /**
    * Find specified DatanodeStorageInfo.
    * @return DatanodeStorageInfo or null if not found.
    */
@@ -372,19 +366,12 @@ public abstract class  BlockInfo extends Block
   }
 
   /**
-   * Convert an under construction block to a complete block.
-   *
-   * @return BlockInfo - a complete block.
-   * @throws IOException if the state of the block
-   * (the generation stamp and the length) has not been committed by
-   * the client or it does not have at least a minimal number of replicas
-   * reported from data-nodes.
+   * Convert an under construction block to complete.
    */
-  BlockInfo convertToCompleteBlock() throws IOException {
+  void convertToCompleteBlock() {
     assert getBlockUCState() != BlockUCState.COMPLETE :
         "Trying to convert a COMPLETE block";
     uc = null;
-    return this;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index 42934c3..94fb222 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -104,19 +104,4 @@ public class BlockInfoContiguous extends BlockInfo {
     }
     return 0;
   }
-
-  @Override
-  void replaceBlock(BlockInfo newBlock) {
-    assert newBlock instanceof BlockInfoContiguous;
-    for (int i = this.numNodes() - 1; i >= 0; i--) {
-      final DatanodeStorageInfo storage = this.getStorageInfo(i);
-      final boolean removed = storage.removeBlock(this);
-      assert removed : "currentBlock not found.";
-
-      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
-          newBlock);
-      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
-          "newBlock already exists.";
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8f7bb55..1346ab3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -648,37 +648,34 @@ public class BlockManager implements BlockStatsMXBean {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if(countNodes(lastBlock).liveReplicas() >= minReplication)
-      completeBlock(bc, bc.numBlocks()-1, false);
+    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+      completeBlock(lastBlock, false);
+    }
     return b;
   }
 
   /**
    * Convert a specified block of the file to a complete block.
-   * @param bc file
-   * @param blkIndex  block index in the file
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private BlockInfo completeBlock(final BlockCollection bc,
-      final int blkIndex, boolean force) throws IOException {
-    if(blkIndex < 0)
-      return null;
-    BlockInfo curBlock = bc.getBlocks()[blkIndex];
-    if(curBlock.isComplete())
-      return curBlock;
+  private void completeBlock(BlockInfo curBlock, boolean force)
+      throws IOException {
+    if (curBlock.isComplete()) {
+      return;
+    }
 
     int numNodes = curBlock.numNodes();
-    if (!force && numNodes < minReplication)
-      throw new IOException("Cannot complete block: " +
-          "block does not satisfy minimal replication requirement.");
-    if(!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED)
+    if (!force && numNodes < minReplication) {
+      throw new IOException("Cannot complete block: "
+          + "block does not satisfy minimal replication requirement.");
+    }
+    if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
-    BlockInfo completeBlock = curBlock.convertToCompleteBlock();
-    // replace penultimate block in file
-    bc.setBlock(blkIndex, completeBlock);
-    
+    }
+
+    curBlock.convertToCompleteBlock();
     // Since safe-mode only counts complete blocks, and we now have
     // one more complete block, we need to adjust the total up, and
     // also count it as safe, if we have at least the minimum replica
@@ -688,33 +685,18 @@ public class BlockManager implements BlockStatsMXBean {
     namesystem.adjustSafeModeBlockTotals(0, 1);
     namesystem.incrementSafeBlockCount(
         Math.min(numNodes, minReplication));
-    
-    // replace block in the blocksMap
-    return blocksMap.replaceBlock(completeBlock);
   }
 
-  private BlockInfo completeBlock(final BlockCollection bc,
-      final BlockInfo block, boolean force) throws IOException {
-    BlockInfo[] fileBlocks = bc.getBlocks();
-    for(int idx = 0; idx < fileBlocks.length; idx++)
-      if(fileBlocks[idx] == block) {
-        return completeBlock(bc, idx, force);
-      }
-    return block;
-  }
-  
   /**
    * Force the given block in the given file to be marked as complete,
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    */
-  public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfo block) throws IOException {
+  public void forceCompleteBlock(final BlockInfo block) throws IOException {
     block.commitBlock(block);
-    return completeBlock(bc, block, true);
+    completeBlock(block, true);
   }
 
-  
   /**
    * Convert the last block of the file to an under construction block.<p>
    * The block is converted only if the file has blocks and the last one
@@ -2503,7 +2485,7 @@ public class BlockManager implements BlockStatsMXBean {
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
         && numCurrentReplica >= minReplication) {
-      completeBlock(getBlockCollection(storedBlock), storedBlock, false);
+      completeBlock(storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that.
@@ -2577,7 +2559,7 @@ public class BlockManager implements BlockStatsMXBean {
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         numLiveReplicas >= minReplication) {
-      storedBlock = completeBlock(bc, storedBlock, false);
+      completeBlock(storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that

http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 33c68f3..9189c32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -209,20 +209,4 @@ class BlocksMap {
   int getCapacity() {
     return capacity;
   }
-
-  /**
-   * Replace a block in the block map by a new block.
-   * The new block and the old one have the same key.
-   * @param newBlock - block for replacement
-   * @return new block
-   */
-  BlockInfo replaceBlock(BlockInfo newBlock) {
-    BlockInfo currentBlock = blocks.get(newBlock);
-    assert currentBlock != null : "the block if not in blocksMap";
-    // replace block in data-node lists
-    currentBlock.replaceBlock(newBlock);
-    // replace block in the map itself
-    blocks.put(newBlock);
-    return newBlock;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/caa04de1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index f22762c..fc0bb78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -506,7 +506,7 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(fsDir, addBlockOp, oldFile);
+      addNewBlock(addBlockOp, oldFile);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -940,7 +940,7 @@ public class FSEditLogLoader {
   /**
    * Add a new block into the given INodeFile
    */
-  private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
+  private void addNewBlock(AddBlockOp op, INodeFile file)
       throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
@@ -960,7 +960,7 @@ public class FSEditLogLoader {
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
       if (!oldLastBlock.isComplete()) {
-        fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock);
+        fsNamesys.getBlockManager().forceCompleteBlock(oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
@@ -1013,7 +1013,7 @@ public class FSEditLogLoader {
       if (!oldBlock.isComplete() &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
-        fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock);
+        fsNamesys.getBlockManager().forceCompleteBlock(oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be


[37/50] [abbrv] hadoop git commit: HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d921507..1e561cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.protocolPB;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
+import org.apache.hadoop.crypto.CipherOption;
+import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -264,4 +266,104 @@ public class PBHelperClient {
     assert size >= 0;
     return new ExactSizeInputStream(input, size);
   }
+
+  public static CipherOption convert(HdfsProtos.CipherOptionProto proto) {
+    if (proto != null) {
+      CipherSuite suite = null;
+      if (proto.getSuite() != null) {
+        suite = convert(proto.getSuite());
+      }
+      byte[] inKey = null;
+      if (proto.getInKey() != null) {
+        inKey = proto.getInKey().toByteArray();
+      }
+      byte[] inIv = null;
+      if (proto.getInIv() != null) {
+        inIv = proto.getInIv().toByteArray();
+      }
+      byte[] outKey = null;
+      if (proto.getOutKey() != null) {
+        outKey = proto.getOutKey().toByteArray();
+      }
+      byte[] outIv = null;
+      if (proto.getOutIv() != null) {
+        outIv = proto.getOutIv().toByteArray();
+      }
+      return new CipherOption(suite, inKey, inIv, outKey, outIv);
+    }
+    return null;
+  }
+
+  public static CipherSuite convert(HdfsProtos.CipherSuiteProto proto) {
+    switch (proto) {
+    case AES_CTR_NOPADDING:
+      return CipherSuite.AES_CTR_NOPADDING;
+    default:
+      // Set to UNKNOWN and stash the unknown enum value
+      CipherSuite suite = CipherSuite.UNKNOWN;
+      suite.setUnknownValue(proto.getNumber());
+      return suite;
+    }
+  }
+
+  public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
+    if (option != null) {
+      HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
+          newBuilder();
+      if (option.getCipherSuite() != null) {
+        builder.setSuite(convert(option.getCipherSuite()));
+      }
+      if (option.getInKey() != null) {
+        builder.setInKey(ByteString.copyFrom(option.getInKey()));
+      }
+      if (option.getInIv() != null) {
+        builder.setInIv(ByteString.copyFrom(option.getInIv()));
+      }
+      if (option.getOutKey() != null) {
+        builder.setOutKey(ByteString.copyFrom(option.getOutKey()));
+      }
+      if (option.getOutIv() != null) {
+        builder.setOutIv(ByteString.copyFrom(option.getOutIv()));
+      }
+      return builder.build();
+    }
+    return null;
+  }
+
+  public static HdfsProtos.CipherSuiteProto convert(CipherSuite suite) {
+    switch (suite) {
+    case UNKNOWN:
+      return HdfsProtos.CipherSuiteProto.UNKNOWN;
+    case AES_CTR_NOPADDING:
+      return HdfsProtos.CipherSuiteProto.AES_CTR_NOPADDING;
+    default:
+      return null;
+    }
+  }
+
+  public static List<HdfsProtos.CipherOptionProto> convertCipherOptions(
+      List<CipherOption> options) {
+    if (options != null) {
+      List<HdfsProtos.CipherOptionProto> protos =
+          Lists.newArrayListWithCapacity(options.size());
+      for (CipherOption option : options) {
+        protos.add(convert(option));
+      }
+      return protos;
+    }
+    return null;
+  }
+
+  public static List<CipherOption> convertCipherOptionProtos(
+      List<HdfsProtos.CipherOptionProto> protos) {
+    if (protos != null) {
+      List<CipherOption> options =
+          Lists.newArrayListWithCapacity(protos.size());
+      for (HdfsProtos.CipherOptionProto proto : protos) {
+        options.add(convert(proto));
+      }
+      return options;
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 42460ed..ce3fbb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -883,6 +883,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8890. Allow admin to specify which blockpools the balancer should run
     on. (Chris Trezzo via mingma)
 
+    HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client.
+    (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 268a5b9..95e9ad4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -99,7 +99,6 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -3018,7 +3017,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       sock = socketFactory.createSocket();
       NetUtils.connect(sock, addr, getRandomLocalInterfaceAddr(), socketTimeout);
-      peer = TcpPeerServer.peerFromSocketAndKey(saslClient, sock, this,
+      peer = DFSUtilClient.peerFromSocketAndKey(saslClient, sock, this,
           blockToken, datanodeId);
       peer.setReadTimeout(socketTimeout);
       peer.setWriteTimeout(socketTimeout);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/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 6420b55..84858f6 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
@@ -599,14 +599,28 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   // Security-related configs
   public static final String DFS_ENCRYPT_DATA_TRANSFER_KEY = "dfs.encrypt.data.transfer";
   public static final boolean DFS_ENCRYPT_DATA_TRANSFER_DEFAULT = false;
-  public static final String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY = "dfs.encrypt.data.transfer.cipher.key.bitlength";
-  public static final int    DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
-  public static final String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY = "dfs.encrypt.data.transfer.cipher.suites";
+  @Deprecated
+  public static final String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY =
+      HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY;
+  @Deprecated
+  public static final int    DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT =
+      HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT;
+  @Deprecated
+  public static final String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY =
+      HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
   public static final String DFS_DATA_ENCRYPTION_ALGORITHM_KEY = "dfs.encrypt.data.transfer.algorithm";
-  public static final String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
-  public static final String DFS_DATA_TRANSFER_PROTECTION_KEY = "dfs.data.transfer.protection";
-  public static final String DFS_DATA_TRANSFER_PROTECTION_DEFAULT = "";
-  public static final String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY = "dfs.data.transfer.saslproperties.resolver.class";
+  @Deprecated
+  public static final String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS =
+      HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS;
+  @Deprecated
+  public static final String DFS_DATA_TRANSFER_PROTECTION_KEY =
+      HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+  @Deprecated
+  public static final String DFS_DATA_TRANSFER_PROTECTION_DEFAULT =
+      HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_DEFAULT;
+  @Deprecated
+  public static final String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY =
+      HdfsClientConfigKeys.DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY;
   public static final int    DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT = 100;
   public static final String DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES = "dfs.namenode.list.encryption.zones.num.responses";
   public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
deleted file mode 100644
index a9f33e7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
+++ /dev/null
@@ -1,133 +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.hdfs.net;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.nio.channels.ReadableByteChannel;
-
-import org.apache.hadoop.net.unix.DomainSocket;
-
-/**
- * Represents a peer that we communicate with by using a basic Socket
- * that has no associated Channel.
- *
- */
-class BasicInetPeer implements Peer {
-  private final Socket socket;
-  private final OutputStream out;
-  private final InputStream in;
-  private final boolean isLocal;
-
-  public BasicInetPeer(Socket socket) throws IOException {
-    this.socket = socket;
-    this.out = socket.getOutputStream();
-    this.in = socket.getInputStream();
-    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
-  }
-
-  @Override
-  public ReadableByteChannel getInputStreamChannel() {
-    /*
-     * This Socket has no channel, so there's nothing to return here.
-     */
-    return null;
-  }
-
-  @Override
-  public void setReadTimeout(int timeoutMs) throws IOException {
-    socket.setSoTimeout(timeoutMs);
-  }
-
-  @Override
-  public int getReceiveBufferSize() throws IOException {
-    return socket.getReceiveBufferSize();
-  }
-
-  @Override
-  public boolean getTcpNoDelay() throws IOException {
-    return socket.getTcpNoDelay();
-  }
-
-  @Override
-  public void setWriteTimeout(int timeoutMs) {
-   /* 
-    * We can't implement write timeouts. :(
-    * 
-    * Java provides no facility to set a blocking write timeout on a Socket.
-    * You can simulate a blocking write with a timeout by using
-    * non-blocking I/O.  However, we can't use nio here, because this Socket
-    * doesn't have an associated Channel.
-    * 
-    * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
-    * more details.
-    */
-  }
-
-  @Override
-  public boolean isClosed() {
-    return socket.isClosed();
-  }
-
-  @Override
-  public void close() throws IOException {
-    socket.close();
-  }
-
-  @Override
-  public String getRemoteAddressString() {
-    return socket.getRemoteSocketAddress().toString();
-  }
-
-  @Override
-  public String getLocalAddressString() {
-    return socket.getLocalSocketAddress().toString();
-  }
-  
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return in;
-  }
-
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return out;
-  }
-
-  @Override
-  public boolean isLocal() {
-    return isLocal;
-  }
-
-  @Override
-  public String toString() {
-    return "BasicInetPeer(" + socket.toString() + ")";
-  }
-
-  @Override
-  public DomainSocket getDomainSocket() {
-    return null;
-  }
-
-  @Override
-  public boolean hasSecureChannel() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
deleted file mode 100644
index da660c7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
+++ /dev/null
@@ -1,142 +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.hdfs.net;
-
-import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.net.unix.DomainSocket;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.channels.ReadableByteChannel;
-
-/**
- * Represents a peer that we communicate with by using an encrypted
- * communications medium.
- */
-@InterfaceAudience.Private
-public class EncryptedPeer implements Peer {
-  private final Peer enclosedPeer;
-
-  /**
-   * An encrypted InputStream.
-   */
-  private final InputStream in;
-  
-  /**
-   * An encrypted OutputStream.
-   */
-  private final OutputStream out;
-  
-  /**
-   * An encrypted ReadableByteChannel.
-   */
-  private final ReadableByteChannel channel;
-
-  public EncryptedPeer(Peer enclosedPeer, IOStreamPair ios) {
-    this.enclosedPeer = enclosedPeer;
-    this.in = ios.in;
-    this.out = ios.out;
-    this.channel = ios.in instanceof ReadableByteChannel ? 
-        (ReadableByteChannel)ios.in : null;
-  }
-
-  @Override
-  public ReadableByteChannel getInputStreamChannel() {
-    return channel;
-  }
-
-  @Override
-  public void setReadTimeout(int timeoutMs) throws IOException {
-    enclosedPeer.setReadTimeout(timeoutMs);
-  }
-
-  @Override
-  public int getReceiveBufferSize() throws IOException {
-    return enclosedPeer.getReceiveBufferSize();
-  }
-
-  @Override
-  public boolean getTcpNoDelay() throws IOException {
-    return enclosedPeer.getTcpNoDelay();
-  }
-
-  @Override
-  public void setWriteTimeout(int timeoutMs) throws IOException {
-    enclosedPeer.setWriteTimeout(timeoutMs);
-  }
-
-  @Override
-  public boolean isClosed() {
-    return enclosedPeer.isClosed();
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      in.close();
-    } finally {
-      try {
-        out.close();
-      } finally {
-        enclosedPeer.close();
-      }
-    }
-  }
-
-  @Override
-  public String getRemoteAddressString() {
-    return enclosedPeer.getRemoteAddressString();
-  }
-
-  @Override
-  public String getLocalAddressString() {
-    return enclosedPeer.getLocalAddressString();
-  }
-
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return in;
-  }
-
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return out;
-  }
-
-  @Override
-  public boolean isLocal() {
-    return enclosedPeer.isLocal();
-  }
-
-  @Override
-  public String toString() {
-    return "EncryptedPeer(" + enclosedPeer + ")";
-  }
-
-  @Override
-  public DomainSocket getDomainSocket() {
-    return enclosedPeer.getDomainSocket();
-  }
-
-  @Override
-  public boolean hasSecureChannel() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
deleted file mode 100644
index 5bb4f56..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
+++ /dev/null
@@ -1,136 +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.hdfs.net;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.nio.channels.ReadableByteChannel;
-
-import org.apache.hadoop.net.SocketInputStream;
-import org.apache.hadoop.net.SocketOutputStream;
-import org.apache.hadoop.net.unix.DomainSocket;
-
-/**
- * Represents a peer that we communicate with by using non-blocking I/O 
- * on a Socket.
- */
-class NioInetPeer implements Peer {
-  private final Socket socket;
-
-  /**
-   * An InputStream which simulates blocking I/O with timeouts using NIO.
-   */
-  private final SocketInputStream in;
-  
-  /**
-   * An OutputStream which simulates blocking I/O with timeouts using NIO.
-   */
-  private final SocketOutputStream out;
-
-  private final boolean isLocal;
-
-  NioInetPeer(Socket socket) throws IOException {
-    this.socket = socket;
-    this.in = new SocketInputStream(socket.getChannel(), 0);
-    this.out = new SocketOutputStream(socket.getChannel(), 0);
-    this.isLocal = socket.getInetAddress().equals(socket.getLocalAddress());
-  }
-
-  @Override
-  public ReadableByteChannel getInputStreamChannel() {
-    return in;
-  }
-
-  @Override
-  public void setReadTimeout(int timeoutMs) throws IOException {
-    in.setTimeout(timeoutMs);
-  }
-
-  @Override
-  public int getReceiveBufferSize() throws IOException {
-    return socket.getReceiveBufferSize();
-  }
-
-  @Override
-  public boolean getTcpNoDelay() throws IOException {
-    return socket.getTcpNoDelay();
-  }
-
-  @Override
-  public void setWriteTimeout(int timeoutMs) throws IOException {
-    out.setTimeout(timeoutMs);
-  }
-
-  @Override
-  public boolean isClosed() {
-    return socket.isClosed();
-  }
-
-  @Override
-  public void close() throws IOException {
-    // We always close the outermost streams-- in this case, 'in' and 'out'
-    // Closing either one of these will also close the Socket.
-    try {
-      in.close();
-    } finally {
-      out.close();
-    }
-  }
-
-  @Override
-  public String getRemoteAddressString() {
-    return socket.getRemoteSocketAddress().toString();
-  }
-
-  @Override
-  public String getLocalAddressString() {
-    return socket.getLocalSocketAddress().toString();
-  }
-
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return in;
-  }
-
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return out;
-  }
-
-  @Override
-  public boolean isLocal() {
-    return isLocal;
-  }
-
-  @Override
-  public String toString() {
-    return "NioInetPeer(" + socket.toString() + ")";
-  }
-
-  @Override
-  public DomainSocket getDomainSocket() {
-    return null;
-  }
-
-  @Override
-  public boolean hasSecureChannel() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java
index 2a547e0..e31e46a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/net/TcpPeerServer.java
@@ -20,22 +20,15 @@ package org.apache.hadoop.hdfs.net;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
-import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.security.token.Token;
 
 @InterfaceAudience.Private
 public class TcpPeerServer implements PeerServer {
@@ -43,60 +36,6 @@ public class TcpPeerServer implements PeerServer {
 
   private final ServerSocket serverSocket;
 
-  public static Peer peerFromSocket(Socket socket)
-      throws IOException {
-    Peer peer = null;
-    boolean success = false;
-    try {
-      // TCP_NODELAY is crucial here because of bad interactions between
-      // Nagle's Algorithm and Delayed ACKs. With connection keepalive
-      // between the client and DN, the conversation looks like:
-      //   1. Client -> DN: Read block X
-      //   2. DN -> Client: data for block X
-      //   3. Client -> DN: Status OK (successful read)
-      //   4. Client -> DN: Read block Y
-      // The fact that step #3 and #4 are both in the client->DN direction
-      // triggers Nagling. If the DN is using delayed ACKs, this results
-      // in a delay of 40ms or more.
-      //
-      // TCP_NODELAY disables nagling and thus avoids this performance
-      // disaster.
-      socket.setTcpNoDelay(true);
-      SocketChannel channel = socket.getChannel();
-      if (channel == null) {
-        peer = new BasicInetPeer(socket);
-      } else {
-        peer = new NioInetPeer(socket);
-      }
-      success = true;
-      return peer;
-    } finally {
-      if (!success) {
-        if (peer != null) peer.close();
-        socket.close();
-      }
-    }
-  }
-
-  public static Peer peerFromSocketAndKey(
-        SaslDataTransferClient saslClient, Socket s,
-        DataEncryptionKeyFactory keyFactory,
-        Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
-        throws IOException {
-    Peer peer = null;
-    boolean success = false;
-    try {
-      peer = peerFromSocket(s);
-      peer = saslClient.peerSend(peer, keyFactory, blockToken, datanodeId);
-      success = true;
-      return peer;
-    } finally {
-      if (!success) {
-        IOUtils.cleanup(null, peer);
-      }
-    }
-  }
-
   /**
    * Create a non-secure TcpPeerServer.
    *
@@ -136,7 +75,7 @@ public class TcpPeerServer implements PeerServer {
 
   @Override
   public Peer accept() throws IOException, SocketTimeoutException {
-    Peer peer = peerFromSocket(serverSocket.accept());
+    Peer peer = DFSUtilClient.peerFromSocket(serverSocket.accept());
     return peer;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
deleted file mode 100644
index 23407f8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
+++ /dev/null
@@ -1,37 +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.hdfs.protocol.datatransfer;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * A little struct class to wrap an InputStream and an OutputStream.
- */
-@InterfaceAudience.Private
-public class IOStreamPair {
-  public final InputStream in;
-  public final OutputStream out;
-  
-  public IOStreamPair(InputStream in, OutputStream out) {
-    this.in = in;
-    this.out = out;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
deleted file mode 100644
index 9e6a43d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
+++ /dev/null
@@ -1,81 +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.hdfs.protocol.datatransfer;
-
-import java.net.InetAddress;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * Class used to indicate whether a channel is trusted or not.
- * The default implementation is to return false indicating that
- * the channel is not trusted.
- * This class can be overridden to provide custom logic to determine
- * whether a channel is trusted or not. 
- * The custom class can be specified via configuration.
- *
- */
-public class TrustedChannelResolver implements Configurable {
-  Configuration conf;
-
-  /**
-   * Returns an instance of TrustedChannelResolver.
-   * Looks up the configuration to see if there is custom class specified.
-   * @param conf
-   * @return TrustedChannelResolver
-   */
-  public static TrustedChannelResolver getInstance(Configuration conf) {
-    Class<? extends TrustedChannelResolver> clazz =
-      conf.getClass(
-          DFSConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
-          TrustedChannelResolver.class, TrustedChannelResolver.class);
-    return ReflectionUtils.newInstance(clazz, conf);
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  /**
-   * Return boolean value indicating whether a channel is trusted or not
-   * from a client's perspective.
-   * @return true if the channel is trusted and false otherwise.
-   */
-  public boolean isTrusted() {
-    return false;
-  }
-
-
-  /**
-   * Identify boolean value indicating whether a channel is trusted or not.
-   * @param peerAddress address of the peer
-   * @return true if the channel is trusted and false otherwise.
-   */
-  public boolean isTrusted(InetAddress peerAddress) {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java
deleted file mode 100644
index 959cba0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataEncryptionKeyFactory.java
+++ /dev/null
@@ -1,38 +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.hdfs.protocol.datatransfer.sasl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-
-/**
- * Creates a new {@link DataEncryptionKey} on demand.
- */
-@InterfaceAudience.Private
-public interface DataEncryptionKeyFactory {
-
-  /**
-   * Creates a new DataEncryptionKey.
-   *
-   * @return DataEncryptionKey newly created
-   * @throws IOException for any error
-   */
-  DataEncryptionKey newDataEncryptionKey() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
deleted file mode 100644
index 852819f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ /dev/null
@@ -1,519 +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.hdfs.protocol.datatransfer.sasl;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
-import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.security.sasl.Sasl;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherOption;
-import org.apache.hadoop.crypto.CipherSuite;
-import org.apache.hadoop.crypto.CryptoCodec;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto.DataTransferEncryptorStatus;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CipherOptionProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.security.SaslPropertiesResolver;
-import org.apache.hadoop.security.SaslRpcServer.QualityOfProtection;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.net.InetAddresses;
-import com.google.protobuf.ByteString;
-
-/**
- * Utility methods implementing SASL negotiation for DataTransferProtocol.
- */
-@InterfaceAudience.Private
-public final class DataTransferSaslUtil {
-
-  private static final Logger LOG = LoggerFactory.getLogger(
-    DataTransferSaslUtil.class);
-
-  /**
-   * Delimiter for the three-part SASL username string.
-   */
-  public static final String NAME_DELIMITER = " ";
-
-  /**
-   * Sent by clients and validated by servers. We use a number that's unlikely
-   * to ever be sent as the value of the DATA_TRANSFER_VERSION.
-   */
-  public static final int SASL_TRANSFER_MAGIC_NUMBER = 0xDEADBEEF;
-
-  /**
-   * Checks that SASL negotiation has completed for the given participant, and
-   * the negotiated quality of protection is included in the given SASL
-   * properties and therefore acceptable.
-   *
-   * @param sasl participant to check
-   * @param saslProps properties of SASL negotiation
-   * @throws IOException for any error
-   */
-  public static void checkSaslComplete(SaslParticipant sasl,
-      Map<String, String> saslProps) throws IOException {
-    if (!sasl.isComplete()) {
-      throw new IOException("Failed to complete SASL handshake");
-    }
-    Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
-      saslProps.get(Sasl.QOP).split(",")));
-    String negotiatedQop = sasl.getNegotiatedQop();
-    LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
-      requestedQop, negotiatedQop);
-    if (!requestedQop.contains(negotiatedQop)) {
-      throw new IOException(String.format("SASL handshake completed, but " +
-        "channel does not have acceptable quality of protection, " +
-        "requested = %s, negotiated = %s", requestedQop, negotiatedQop));
-    }
-  }
-  
-  /**
-   * Check whether requested SASL Qop contains privacy.
-   * 
-   * @param saslProps properties of SASL negotiation
-   * @return boolean true if privacy exists
-   */
-  public static boolean requestedQopContainsPrivacy(
-      Map<String, String> saslProps) {
-    Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
-        saslProps.get(Sasl.QOP).split(",")));
-    return requestedQop.contains("auth-conf");
-  }
-
-  /**
-   * Creates SASL properties required for an encrypted SASL negotiation.
-   *
-   * @param encryptionAlgorithm to use for SASL negotation
-   * @return properties of encrypted SASL negotiation
-   */
-  public static Map<String, String> createSaslPropertiesForEncryption(
-      String encryptionAlgorithm) {
-    Map<String, String> saslProps = Maps.newHashMapWithExpectedSize(3);
-    saslProps.put(Sasl.QOP, QualityOfProtection.PRIVACY.getSaslQop());
-    saslProps.put(Sasl.SERVER_AUTH, "true");
-    saslProps.put("com.sun.security.sasl.digest.cipher", encryptionAlgorithm);
-    return saslProps;
-  }
-
-  /**
-   * For an encrypted SASL negotiation, encodes an encryption key to a SASL
-   * password.
-   *
-   * @param encryptionKey to encode
-   * @return key encoded as SASL password
-   */
-  public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
-    return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
-      .toCharArray();
-  }
-
-  /**
-   * Returns InetAddress from peer.  The getRemoteAddressString has the form
-   * [host][/ip-address]:port.  The host may be missing.  The IP address (and
-   * preceding '/') may be missing.  The port preceded by ':' is always present.
-   *
-   * @param peer
-   * @return InetAddress from peer
-   */
-  public static InetAddress getPeerAddress(Peer peer) {
-    String remoteAddr = peer.getRemoteAddressString().split(":")[0];
-    int slashIdx = remoteAddr.indexOf('/');
-    return InetAddresses.forString(slashIdx != -1 ?
-        remoteAddr.substring(slashIdx + 1, remoteAddr.length()) :
-        remoteAddr);
-  }
-
-  /**
-   * Creates a SaslPropertiesResolver from the given configuration.  This method
-   * works by cloning the configuration, translating configuration properties
-   * specific to DataTransferProtocol to what SaslPropertiesResolver expects,
-   * and then delegating to SaslPropertiesResolver for initialization.  This
-   * method returns null if SASL protection has not been configured for
-   * DataTransferProtocol.
-   *
-   * @param conf configuration to read
-   * @return SaslPropertiesResolver for DataTransferProtocol, or null if not
-   *   configured
-   */
-  public static SaslPropertiesResolver getSaslPropertiesResolver(
-      Configuration conf) {
-    String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
-    if (qops == null || qops.isEmpty()) {
-      LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
-        "QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
-      return null;
-    }
-    Configuration saslPropsResolverConf = new Configuration(conf);
-    saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
-    Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
-      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
-      SaslPropertiesResolver.class, SaslPropertiesResolver.class);
-    resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
-      resolverClass, SaslPropertiesResolver.class);
-    saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
-      resolverClass, SaslPropertiesResolver.class);
-    SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
-      saslPropsResolverConf);
-    LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
-      "QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops, 
-      DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
-    return resolver;
-  }
-
-  /**
-   * Reads a SASL negotiation message.
-   *
-   * @param in stream to read
-   * @return bytes of SASL negotiation messsage
-   * @throws IOException for any error
-   */
-  public static byte[] readSaslMessage(InputStream in) throws IOException {
-    DataTransferEncryptorMessageProto proto =
-        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
-    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
-      throw new InvalidEncryptionKeyException(proto.getMessage());
-    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
-      throw new IOException(proto.getMessage());
-    } else {
-      return proto.getPayload().toByteArray();
-    }
-  }
-  
-  /**
-   * Reads a SASL negotiation message and negotiation cipher options. 
-   * 
-   * @param in stream to read
-   * @param cipherOptions list to store negotiation cipher options
-   * @return byte[] SASL negotiation message
-   * @throws IOException for any error
-   */
-  public static byte[] readSaslMessageAndNegotiationCipherOptions(
-      InputStream in, List<CipherOption> cipherOptions) throws IOException {
-    DataTransferEncryptorMessageProto proto =
-        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
-    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
-      throw new InvalidEncryptionKeyException(proto.getMessage());
-    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
-      throw new IOException(proto.getMessage());
-    } else {
-      List<CipherOptionProto> optionProtos = proto.getCipherOptionList();
-      if (optionProtos != null) {
-        for (CipherOptionProto optionProto : optionProtos) {
-          cipherOptions.add(PBHelper.convert(optionProto));
-        }
-      }
-      return proto.getPayload().toByteArray();
-    }
-  }
-  
-  /**
-   * Negotiate a cipher option which server supports.
-   * 
-   * @param conf the configuration
-   * @param options the cipher options which client supports
-   * @return CipherOption negotiated cipher option
-   */
-  public static CipherOption negotiateCipherOption(Configuration conf,
-      List<CipherOption> options) throws IOException {
-    // Negotiate cipher suites if configured.  Currently, the only supported
-    // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
-    // values for future expansion.
-    String cipherSuites = conf.get(DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
-    if (cipherSuites == null || cipherSuites.isEmpty()) {
-      return null;
-    }
-    if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
-      throw new IOException(String.format("Invalid cipher suite, %s=%s",
-          DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
-    }
-    if (options != null) {
-      for (CipherOption option : options) {
-        CipherSuite suite = option.getCipherSuite();
-        if (suite == CipherSuite.AES_CTR_NOPADDING) {
-          int keyLen = conf.getInt(
-              DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY,
-              DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT) / 8;
-          CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
-          byte[] inKey = new byte[keyLen];
-          byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
-          byte[] outKey = new byte[keyLen];
-          byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
-          codec.generateSecureRandom(inKey);
-          codec.generateSecureRandom(inIv);
-          codec.generateSecureRandom(outKey);
-          codec.generateSecureRandom(outIv);
-          return new CipherOption(suite, inKey, inIv, outKey, outIv);
-        }
-      }
-    }
-    return null;
-  }
-  
-  /**
-   * Send SASL message and negotiated cipher option to client.
-   * 
-   * @param out stream to receive message
-   * @param payload to send
-   * @param option negotiated cipher option
-   * @throws IOException for any error
-   */
-  public static void sendSaslMessageAndNegotiatedCipherOption(
-      OutputStream out, byte[] payload, CipherOption option) 
-          throws IOException {
-    DataTransferEncryptorMessageProto.Builder builder =
-        DataTransferEncryptorMessageProto.newBuilder();
-    
-    builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
-    if (payload != null) {
-      builder.setPayload(ByteString.copyFrom(payload));
-    }
-    if (option != null) {
-      builder.addCipherOption(PBHelper.convert(option));
-    }
-    
-    DataTransferEncryptorMessageProto proto = builder.build();
-    proto.writeDelimitedTo(out);
-    out.flush();
-  }
-  
-  /**
-   * Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
-   * and {@link org.apache.hadoop.crypto.CryptoOutputStream}
-   * 
-   * @param conf the configuration
-   * @param cipherOption negotiated cipher option
-   * @param out underlying output stream
-   * @param in underlying input stream
-   * @param isServer is server side
-   * @return IOStreamPair the stream pair
-   * @throws IOException for any error
-   */
-  public static IOStreamPair createStreamPair(Configuration conf,
-      CipherOption cipherOption, OutputStream out, InputStream in, 
-      boolean isServer) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
-          "CryptoOutputStream.");
-    }
-    CryptoCodec codec = CryptoCodec.getInstance(conf, 
-        cipherOption.getCipherSuite());
-    byte[] inKey = cipherOption.getInKey();
-    byte[] inIv = cipherOption.getInIv();
-    byte[] outKey = cipherOption.getOutKey();
-    byte[] outIv = cipherOption.getOutIv();
-    InputStream cIn = new CryptoInputStream(in, codec, 
-        isServer ? inKey : outKey, isServer ? inIv : outIv);
-    OutputStream cOut = new CryptoOutputStream(out, codec, 
-        isServer ? outKey : inKey, isServer ? outIv : inIv);
-    return new IOStreamPair(cIn, cOut);
-  }
-
-  /**
-   * Sends a SASL negotiation message indicating an error.
-   *
-   * @param out stream to receive message
-   * @param message to send
-   * @throws IOException for any error
-   */
-  public static void sendGenericSaslErrorMessage(OutputStream out,
-      String message) throws IOException {
-    sendSaslMessage(out, DataTransferEncryptorStatus.ERROR, null, message);
-  }
-
-  /**
-   * Sends a SASL negotiation message.
-   *
-   * @param out stream to receive message
-   * @param payload to send
-   * @throws IOException for any error
-   */
-  public static void sendSaslMessage(OutputStream out, byte[] payload)
-      throws IOException {
-    sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
-  }
-  
-  /**
-   * Send a SASL negotiation message and negotiation cipher options to server.
-   * 
-   * @param out stream to receive message
-   * @param payload to send
-   * @param options cipher options to negotiate
-   * @throws IOException for any error
-   */
-  public static void sendSaslMessageAndNegotiationCipherOptions(
-      OutputStream out, byte[] payload, List<CipherOption> options)
-          throws IOException {
-    DataTransferEncryptorMessageProto.Builder builder =
-        DataTransferEncryptorMessageProto.newBuilder();
-    
-    builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
-    if (payload != null) {
-      builder.setPayload(ByteString.copyFrom(payload));
-    }
-    if (options != null) {
-      builder.addAllCipherOption(PBHelper.convertCipherOptions(options));
-    }
-    
-    DataTransferEncryptorMessageProto proto = builder.build();
-    proto.writeDelimitedTo(out);
-    out.flush();
-  }
-  
-  /**
-   * Read SASL message and negotiated cipher option from server.
-   * 
-   * @param in stream to read
-   * @return SaslResponseWithNegotiatedCipherOption SASL message and 
-   * negotiated cipher option
-   * @throws IOException for any error
-   */
-  public static SaslResponseWithNegotiatedCipherOption
-      readSaslMessageAndNegotiatedCipherOption(InputStream in)
-          throws IOException {
-    DataTransferEncryptorMessageProto proto =
-        DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
-    if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
-      throw new InvalidEncryptionKeyException(proto.getMessage());
-    } else if (proto.getStatus() == DataTransferEncryptorStatus.ERROR) {
-      throw new IOException(proto.getMessage());
-    } else {
-      byte[] response = proto.getPayload().toByteArray();
-      List<CipherOption> options = PBHelper.convertCipherOptionProtos(
-          proto.getCipherOptionList());
-      CipherOption option = null;
-      if (options != null && !options.isEmpty()) {
-        option = options.get(0);
-      }
-      return new SaslResponseWithNegotiatedCipherOption(response, option);
-    }
-  }
-  
-  /**
-   * Encrypt the key and iv of the negotiated cipher option.
-   * 
-   * @param option negotiated cipher option
-   * @param sasl SASL participant representing server
-   * @return CipherOption negotiated cipher option which contains the 
-   * encrypted key and iv
-   * @throws IOException for any error
-   */
-  public static CipherOption wrap(CipherOption option, SaslParticipant sasl) 
-      throws IOException {
-    if (option != null) {
-      byte[] inKey = option.getInKey();
-      if (inKey != null) {
-        inKey = sasl.wrap(inKey, 0, inKey.length);
-      }
-      byte[] outKey = option.getOutKey();
-      if (outKey != null) {
-        outKey = sasl.wrap(outKey, 0, outKey.length);
-      }
-      return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
-          outKey, option.getOutIv());
-    }
-    
-    return null;
-  }
-  
-  /**
-   * Decrypt the key and iv of the negotiated cipher option.
-   * 
-   * @param option negotiated cipher option
-   * @param sasl SASL participant representing client
-   * @return CipherOption negotiated cipher option which contains the 
-   * decrypted key and iv
-   * @throws IOException for any error
-   */
-  public static CipherOption unwrap(CipherOption option, SaslParticipant sasl)
-      throws IOException {
-    if (option != null) {
-      byte[] inKey = option.getInKey();
-      if (inKey != null) {
-        inKey = sasl.unwrap(inKey, 0, inKey.length);
-      }
-      byte[] outKey = option.getOutKey();
-      if (outKey != null) {
-        outKey = sasl.unwrap(outKey, 0, outKey.length);
-      }
-      return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
-          outKey, option.getOutIv());
-    }
-    
-    return null;
-  }
-
-  /**
-   * Sends a SASL negotiation message.
-   *
-   * @param out stream to receive message
-   * @param status negotiation status
-   * @param payload to send
-   * @param message to send
-   * @throws IOException for any error
-   */
-  public static void sendSaslMessage(OutputStream out,
-      DataTransferEncryptorStatus status, byte[] payload, String message)
-          throws IOException {
-    DataTransferEncryptorMessageProto.Builder builder =
-        DataTransferEncryptorMessageProto.newBuilder();
-    
-    builder.setStatus(status);
-    if (payload != null) {
-      builder.setPayload(ByteString.copyFrom(payload));
-    }
-    if (message != null) {
-      builder.setMessage(message);
-    }
-    
-    DataTransferEncryptorMessageProto proto = builder.build();
-    proto.writeDelimitedTo(out);
-    out.flush();
-  }
-
-  /**
-   * There is no reason to instantiate this class.
-   */
-  private DataTransferSaslUtil() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
deleted file mode 100644
index 00b131f..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
+++ /dev/null
@@ -1,498 +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.hdfs.protocol.datatransfer.sasl;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
-import static org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.*;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.callback.NameCallback;
-import javax.security.auth.callback.PasswordCallback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.RealmCallback;
-import javax.security.sasl.RealmChoiceCallback;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherOption;
-import org.apache.hadoop.crypto.CipherSuite;
-import org.apache.hadoop.hdfs.net.EncryptedPeer;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.security.SaslPropertiesResolver;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.Lists;
-
-/**
- * Negotiates SASL for DataTransferProtocol on behalf of a client.  There are
- * two possible supported variants of SASL negotiation: either a general-purpose
- * negotiation supporting any quality of protection, or a specialized
- * negotiation that enforces privacy as the quality of protection using a
- * cryptographically strong encryption key.
- *
- * This class is used in both the HDFS client and the DataNode.  The DataNode
- * needs it, because it acts as a client to other DataNodes during write
- * pipelines and block transfers.
- */
-@InterfaceAudience.Private
-public class SaslDataTransferClient {
-
-  private static final Logger LOG = LoggerFactory.getLogger(
-    SaslDataTransferClient.class);
-
-  private final Configuration conf;
-  private final AtomicBoolean fallbackToSimpleAuth;
-  private final SaslPropertiesResolver saslPropsResolver;
-  private final TrustedChannelResolver trustedChannelResolver;
-
-  /**
-   * Creates a new SaslDataTransferClient.  This constructor is used in cases
-   * where it is not relevant to track if a secure client did a fallback to
-   * simple auth.  For intra-cluster connections between data nodes in the same
-   * cluster, we can assume that all run under the same security configuration.
-   *
-   * @param conf the configuration
-   * @param saslPropsResolver for determining properties of SASL negotiation
-   * @param trustedChannelResolver for identifying trusted connections that do
-   *   not require SASL negotiation
-   */
-  public SaslDataTransferClient(Configuration conf, 
-      SaslPropertiesResolver saslPropsResolver,
-      TrustedChannelResolver trustedChannelResolver) {
-    this(conf, saslPropsResolver, trustedChannelResolver, null);
-  }
-
-  /**
-   * Creates a new SaslDataTransferClient.
-   *
-   * @param conf the configuration
-   * @param saslPropsResolver for determining properties of SASL negotiation
-   * @param trustedChannelResolver for identifying trusted connections that do
-   *   not require SASL negotiation
-   * @param fallbackToSimpleAuth checked on each attempt at general SASL
-   *   handshake, if true forces use of simple auth
-   */
-  public SaslDataTransferClient(Configuration conf, 
-      SaslPropertiesResolver saslPropsResolver,
-      TrustedChannelResolver trustedChannelResolver,
-      AtomicBoolean fallbackToSimpleAuth) {
-    this.conf = conf;
-    this.fallbackToSimpleAuth = fallbackToSimpleAuth;
-    this.saslPropsResolver = saslPropsResolver;
-    this.trustedChannelResolver = trustedChannelResolver;
-  }
-
-  /**
-   * Sends client SASL negotiation for a newly allocated socket if required.
-   *
-   * @param socket connection socket
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param encryptionKeyFactory for creation of an encryption key
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  public IOStreamPair newSocketSend(Socket socket, OutputStream underlyingOut,
-      InputStream underlyingIn, DataEncryptionKeyFactory encryptionKeyFactory,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    // The encryption key factory only returns a key if encryption is enabled.
-    DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
-      encryptionKeyFactory.newDataEncryptionKey() : null;
-    IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKey, accessToken, datanodeId);
-    return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
-  }
-
-  /**
-   * Sends client SASL negotiation for a peer if required.
-   *
-   * @param peer connection peer
-   * @param encryptionKeyFactory for creation of an encryption key
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  public Peer peerSend(Peer peer, DataEncryptionKeyFactory encryptionKeyFactory,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
-      peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
-      accessToken, datanodeId);
-    // TODO: Consider renaming EncryptedPeer to SaslPeer.
-    return ios != null ? new EncryptedPeer(peer, ios) : peer;
-  }
-
-  /**
-   * Sends client SASL negotiation for a socket if required.
-   *
-   * @param socket connection socket
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param encryptionKeyFactory for creation of an encryption key
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  public IOStreamPair socketSend(Socket socket, OutputStream underlyingOut,
-      InputStream underlyingIn, DataEncryptionKeyFactory encryptionKeyFactory,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
-    return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
-  }
-
-  /**
-   * Checks if an address is already trusted and then sends client SASL
-   * negotiation if required.
-   *
-   * @param addr connection address
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param encryptionKeyFactory for creation of an encryption key
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  private IOStreamPair checkTrustAndSend(InetAddress addr,
-      OutputStream underlyingOut, InputStream underlyingIn,
-      DataEncryptionKeyFactory encryptionKeyFactory,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    if (!trustedChannelResolver.isTrusted() &&
-        !trustedChannelResolver.isTrusted(addr)) {
-      // The encryption key factory only returns a key if encryption is enabled.
-      DataEncryptionKey encryptionKey =
-        encryptionKeyFactory.newDataEncryptionKey();
-      return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
-        datanodeId);
-    } else {
-      LOG.debug(
-        "SASL client skipping handshake on trusted connection for addr = {}, "
-        + "datanodeId = {}", addr, datanodeId);
-      return null;
-    }
-  }
-
-  /**
-   * Sends client SASL negotiation if required.  Determines the correct type of
-   * SASL handshake based on configuration.
-   *
-   * @param addr connection address
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param encryptionKey for an encrypted SASL handshake
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  private IOStreamPair send(InetAddress addr, OutputStream underlyingOut,
-      InputStream underlyingIn, DataEncryptionKey encryptionKey,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    if (encryptionKey != null) {
-      LOG.debug(
-        "SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
-        addr, datanodeId);
-      return getEncryptedStreams(underlyingOut, underlyingIn,
-        encryptionKey);
-    } else if (!UserGroupInformation.isSecurityEnabled()) {
-      LOG.debug(
-        "SASL client skipping handshake in unsecured configuration for "
-        + "addr = {}, datanodeId = {}", addr, datanodeId);
-      return null;
-    } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
-      LOG.debug(
-        "SASL client skipping handshake in secured configuration with "
-        + "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
-      return null;
-    } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
-      LOG.debug(
-        "SASL client skipping handshake in secured configuration with "
-        + "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
-      return null;
-    } else if (saslPropsResolver != null) {
-      LOG.debug(
-        "SASL client doing general handshake for addr = {}, datanodeId = {}",
-        addr, datanodeId);
-      return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
-        datanodeId);
-    } else {
-      // It's a secured cluster using non-privileged ports, but no SASL.  The
-      // only way this can happen is if the DataNode has
-      // ignore.secure.ports.for.testing configured, so this is a rare edge case.
-      LOG.debug(
-        "SASL client skipping handshake in secured configuration with no SASL "
-        + "protection configured for addr = {}, datanodeId = {}",
-        addr, datanodeId);
-      return null;
-    }
-  }
-
-  /**
-   * Sends client SASL negotiation for specialized encrypted handshake.
-   *
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param encryptionKey for an encrypted SASL handshake
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  private IOStreamPair getEncryptedStreams(OutputStream underlyingOut,
-      InputStream underlyingIn, DataEncryptionKey encryptionKey)
-      throws IOException {
-    Map<String, String> saslProps = createSaslPropertiesForEncryption(
-      encryptionKey.encryptionAlgorithm);
-
-    LOG.debug("Client using encryption algorithm {}",
-      encryptionKey.encryptionAlgorithm);
-
-    String userName = getUserNameFromEncryptionKey(encryptionKey);
-    char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
-    CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
-    return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
-  }
-
-  /**
-   * The SASL username for an encrypted handshake consists of the keyId,
-   * blockPoolId, and nonce with the first two encoded as Strings, and the third
-   * encoded using Base64. The fields are each separated by a single space.
-   * 
-   * @param encryptionKey the encryption key to encode as a SASL username.
-   * @return encoded username containing keyId, blockPoolId, and nonce
-   */
-  private static String getUserNameFromEncryptionKey(
-      DataEncryptionKey encryptionKey) {
-    return encryptionKey.keyId + NAME_DELIMITER +
-        encryptionKey.blockPoolId + NAME_DELIMITER +
-        new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
-  }
-
-  /**
-   * Sets user name and password when asked by the client-side SASL object.
-   */
-  private static final class SaslClientCallbackHandler
-      implements CallbackHandler {
-
-    private final char[] password;
-    private final String userName;
-
-    /**
-     * Creates a new SaslClientCallbackHandler.
-     *
-     * @param userName SASL user name
-     * @Param password SASL password
-     */
-    public SaslClientCallbackHandler(String userName, char[] password) {
-      this.password = password;
-      this.userName = userName;
-    }
-
-    @Override
-    public void handle(Callback[] callbacks) throws IOException,
-        UnsupportedCallbackException {
-      NameCallback nc = null;
-      PasswordCallback pc = null;
-      RealmCallback rc = null;
-      for (Callback callback : callbacks) {
-        if (callback instanceof RealmChoiceCallback) {
-          continue;
-        } else if (callback instanceof NameCallback) {
-          nc = (NameCallback) callback;
-        } else if (callback instanceof PasswordCallback) {
-          pc = (PasswordCallback) callback;
-        } else if (callback instanceof RealmCallback) {
-          rc = (RealmCallback) callback;
-        } else {
-          throw new UnsupportedCallbackException(callback,
-              "Unrecognized SASL client callback");
-        }
-      }
-      if (nc != null) {
-        nc.setName(userName);
-      }
-      if (pc != null) {
-        pc.setPassword(password);
-      }
-      if (rc != null) {
-        rc.setText(rc.getDefaultText());
-      }
-    }
-  }
-
-  /**
-   * Sends client SASL negotiation for general-purpose handshake.
-   *
-   * @param addr connection address
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  private IOStreamPair getSaslStreams(InetAddress addr,
-      OutputStream underlyingOut, InputStream underlyingIn,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
-      throws IOException {
-    Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
-
-    String userName = buildUserName(accessToken);
-    char[] password = buildClientPassword(accessToken);
-    CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
-    return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
-  }
-
-  /**
-   * Builds the client's user name for the general-purpose handshake, consisting
-   * of the base64-encoded serialized block access token identifier.  Note that
-   * this includes only the token identifier, not the token itself, which would
-   * include the password.  The password is a shared secret, and we must not
-   * write it on the network during the SASL authentication exchange.
-   *
-   * @param blockToken for block access
-   * @return SASL user name
-   */
-  private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
-    return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
-      Charsets.UTF_8);
-  }
-
-  /**
-   * Calculates the password on the client side for the general-purpose
-   * handshake.  The password consists of the block access token's password.
-   *
-   * @param blockToken for block access
-   * @return SASL password
-   */    
-  private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
-    return new String(Base64.encodeBase64(blockToken.getPassword(), false),
-      Charsets.UTF_8).toCharArray();
-  }
-
-  /**
-   * This method actually executes the client-side SASL handshake.
-   *
-   * @param underlyingOut connection output stream
-   * @param underlyingIn connection input stream
-   * @param userName SASL user name
-   * @param saslProps properties of SASL negotiation
-   * @param callbackHandler for responding to SASL callbacks
-   * @return new pair of streams, wrapped after SASL negotiation
-   * @throws IOException for any error
-   */
-  private IOStreamPair doSaslHandshake(OutputStream underlyingOut,
-      InputStream underlyingIn, String userName, Map<String, String> saslProps,
-      CallbackHandler callbackHandler) throws IOException {
-
-    DataOutputStream out = new DataOutputStream(underlyingOut);
-    DataInputStream in = new DataInputStream(underlyingIn);
-
-    SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
-      saslProps, callbackHandler);
-
-    out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
-    out.flush();
-
-    try {
-      // Start of handshake - "initial response" in SASL terminology.
-      sendSaslMessage(out, new byte[0]);
-
-      // step 1
-      byte[] remoteResponse = readSaslMessage(in);
-      byte[] localResponse = sasl.evaluateChallengeOrResponse(remoteResponse);
-      List<CipherOption> cipherOptions = null;
-      if (requestedQopContainsPrivacy(saslProps)) {
-        // Negotiate cipher suites if configured.  Currently, the only supported
-        // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
-        // values for future expansion.
-        String cipherSuites = conf.get(
-            DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
-        if (cipherSuites != null && !cipherSuites.isEmpty()) {
-          if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
-            throw new IOException(String.format("Invalid cipher suite, %s=%s",
-                DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
-          }
-          CipherOption option = new CipherOption(CipherSuite.AES_CTR_NOPADDING);
-          cipherOptions = Lists.newArrayListWithCapacity(1);
-          cipherOptions.add(option);
-        }
-      }
-      sendSaslMessageAndNegotiationCipherOptions(out, localResponse, 
-          cipherOptions);
-
-      // step 2 (client-side only)
-      SaslResponseWithNegotiatedCipherOption response = 
-          readSaslMessageAndNegotiatedCipherOption(in);
-      localResponse = sasl.evaluateChallengeOrResponse(response.payload);
-      assert localResponse == null;
-
-      // SASL handshake is complete
-      checkSaslComplete(sasl, saslProps);
-
-      CipherOption cipherOption = null;
-      if (sasl.isNegotiatedQopPrivacy()) {
-        // Unwrap the negotiated cipher option
-        cipherOption = unwrap(response.cipherOption, sasl);
-      }
-
-      // If negotiated cipher option is not null, we will use it to create 
-      // stream pair.
-      return cipherOption != null ? createStreamPair(
-          conf, cipherOption, underlyingOut, underlyingIn, false) : 
-            sasl.createStreamPair(out, in);
-    } catch (IOException ioe) {
-      sendGenericSaslErrorMessage(out, ioe.getMessage());
-      throw ioe;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
index f060beb..95965b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol.datatransfer.sasl;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.*;
 
 import java.io.ByteArrayInputStream;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
deleted file mode 100644
index f14a075..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
+++ /dev/null
@@ -1,210 +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.hdfs.protocol.datatransfer.sasl;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.util.Map;
-import javax.security.auth.callback.CallbackHandler;
-import javax.security.sasl.Sasl;
-import javax.security.sasl.SaslClient;
-import javax.security.sasl.SaslException;
-import javax.security.sasl.SaslServer;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.security.SaslInputStream;
-import org.apache.hadoop.security.SaslOutputStream;
-
-/**
- * Strongly inspired by Thrift's TSaslTransport class.
- *
- * Used to abstract over the <code>SaslServer</code> and
- * <code>SaslClient</code> classes, which share a lot of their interface, but
- * unfortunately don't share a common superclass.
- */
-@InterfaceAudience.Private
-class SaslParticipant {
-
-  // This has to be set as part of the SASL spec, but it don't matter for
-  // our purposes, but may not be empty. It's sent over the wire, so use
-  // a short string.
-  private static final String SERVER_NAME = "0";
-  private static final String PROTOCOL = "hdfs";
-  private static final String MECHANISM = "DIGEST-MD5";
-
-  // One of these will always be null.
-  private final SaslServer saslServer;
-  private final SaslClient saslClient;
-
-  /**
-   * Creates a SaslParticipant wrapping a SaslServer.
-   *
-   * @param saslProps properties of SASL negotiation
-   * @param callbackHandler for handling all SASL callbacks
-   * @return SaslParticipant wrapping SaslServer
-   * @throws SaslException for any error
-   */
-  public static SaslParticipant createServerSaslParticipant(
-      Map<String, String> saslProps, CallbackHandler callbackHandler)
-      throws SaslException {
-    return new SaslParticipant(Sasl.createSaslServer(MECHANISM,
-      PROTOCOL, SERVER_NAME, saslProps, callbackHandler));
-  }
-
-  /**
-   * Creates a SaslParticipant wrapping a SaslClient.
-   *
-   * @param userName SASL user name
-   * @param saslProps properties of SASL negotiation
-   * @param callbackHandler for handling all SASL callbacks
-   * @return SaslParticipant wrapping SaslClient
-   * @throws SaslException for any error
-   */
-  public static SaslParticipant createClientSaslParticipant(String userName,
-      Map<String, String> saslProps, CallbackHandler callbackHandler)
-      throws SaslException {
-    return new SaslParticipant(Sasl.createSaslClient(new String[] { MECHANISM },
-      userName, PROTOCOL, SERVER_NAME, saslProps, callbackHandler));
-  }
-
-  /**
-   * Private constructor wrapping a SaslServer.
-   *
-   * @param saslServer to wrap
-   */
-  private SaslParticipant(SaslServer saslServer) {
-    this.saslServer = saslServer;
-    this.saslClient = null;
-  }
-
-  /**
-   * Private constructor wrapping a SaslClient.
-   *
-   * @param saslClient to wrap
-   */
-  private SaslParticipant(SaslClient saslClient) {
-    this.saslServer = null;
-    this.saslClient = saslClient;
-  }
-
-  /**
-   * @see {@link SaslServer#evaluateResponse}
-   * @see {@link SaslClient#evaluateChallenge}
-   */
-  public byte[] evaluateChallengeOrResponse(byte[] challengeOrResponse)
-      throws SaslException {
-    if (saslClient != null) {
-      return saslClient.evaluateChallenge(challengeOrResponse);
-    } else {
-      return saslServer.evaluateResponse(challengeOrResponse);
-    }
-  }
-
-  /**
-   * After successful SASL negotation, returns the negotiated quality of
-   * protection.
-   *
-   * @return negotiated quality of protection
-   */
-  public String getNegotiatedQop() {
-    if (saslClient != null) {
-      return (String) saslClient.getNegotiatedProperty(Sasl.QOP);
-    } else {
-      return (String) saslServer.getNegotiatedProperty(Sasl.QOP);
-    }
-  }
-  
-  /**
-   * After successful SASL negotiation, returns whether it's QOP privacy
-   * 
-   * @return boolean whether it's QOP privacy
-   */
-  public boolean isNegotiatedQopPrivacy() {
-    String qop = getNegotiatedQop();
-    return qop != null && "auth-conf".equalsIgnoreCase(qop);
-  }
-  
-  /**
-   * Wraps a byte array.
-   * 
-   * @param bytes The array containing the bytes to wrap.
-   * @param off The starting position at the array
-   * @param len The number of bytes to wrap
-   * @return byte[] wrapped bytes
-   * @throws SaslException if the bytes cannot be successfully wrapped
-   */
-  public byte[] wrap(byte[] bytes, int off, int len) throws SaslException {
-    if (saslClient != null) {
-      return saslClient.wrap(bytes, off, len);
-    } else {
-      return saslServer.wrap(bytes, off, len);
-    }
-  }
-  
-  /**
-   * Unwraps a byte array.
-   * 
-   * @param bytes The array containing the bytes to unwrap.
-   * @param off The starting position at the array
-   * @param len The number of bytes to unwrap
-   * @return byte[] unwrapped bytes
-   * @throws SaslException if the bytes cannot be successfully unwrapped
-   */
-  public byte[] unwrap(byte[] bytes, int off, int len) throws SaslException {
-    if (saslClient != null) {
-      return saslClient.unwrap(bytes, off, len);
-    } else {
-      return saslServer.unwrap(bytes, off, len);
-    }
-  }
-
-  /**
-   * Returns true if SASL negotiation is complete.
-   *
-   * @return true if SASL negotiation is complete
-   */
-  public boolean isComplete() {
-    if (saslClient != null) {
-      return saslClient.isComplete();
-    } else {
-      return saslServer.isComplete();
-    }
-  }
-
-  /**
-   * Return some input/output streams that may henceforth have their
-   * communication encrypted, depending on the negotiated quality of protection.
-   *
-   * @param out output stream to wrap
-   * @param in input stream to wrap
-   * @return IOStreamPair wrapping the streams
-   */
-  public IOStreamPair createStreamPair(DataOutputStream out,
-      DataInputStream in) {
-    if (saslClient != null) {
-      return new IOStreamPair(
-          new SaslInputStream(in, saslClient),
-          new SaslOutputStream(out, saslClient));
-    } else {
-      return new IOStreamPair(
-          new SaslInputStream(in, saslServer),
-          new SaslOutputStream(out, saslServer));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed78b14e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
deleted file mode 100644
index f69441b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
+++ /dev/null
@@ -1,33 +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.hdfs.protocol.datatransfer.sasl;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.crypto.CipherOption;
-
-@InterfaceAudience.Private
-public class SaslResponseWithNegotiatedCipherOption {
-  final byte[] payload;
-  final CipherOption cipherOption;
-  
-  public SaslResponseWithNegotiatedCipherOption(byte[] payload, 
-      CipherOption cipherOption) {
-    this.payload = payload;
-    this.cipherOption = cipherOption;
-  }
-}
\ No newline at end of file


[35/50] [abbrv] hadoop git commit: HDFS-8939. Test(S)WebHdfsFileContextMainOperations failing on branch-2. Contributed by Chris Nauroth.

Posted by wa...@apache.org.
HDFS-8939. Test(S)WebHdfsFileContextMainOperations failing on branch-2. Contributed by Chris Nauroth.

(cherry picked from commit 9871f57d41990b2763d4c23d9040aeb685b4a80f)


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

Branch: refs/heads/YARN-1197
Commit: c2d2c1802a11e3e11a953b23b0eccbf4d107de59
Parents: 53c38cc
Author: Jakob Homan <jg...@gmail.com>
Authored: Thu Sep 3 13:25:53 2015 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Thu Sep 3 13:26:54 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt          |  3 +++
 .../src/main/java/org/apache/hadoop/fs/SWebHdfs.java | 15 ++++++++++++++-
 .../src/main/java/org/apache/hadoop/fs/WebHdfs.java  | 14 +++++++++++++-
 3 files changed, 30 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2d2c180/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index afc6cf4..42460ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1292,6 +1292,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8964. When validating the edit log, do not read at or beyond the file
     offset that is being written (Zhe Zhang via Colin P. McCabe)
 
+    HDFS-8939. Test(S)WebHdfsFileContextMainOperations failing on branch-2.
+    (Chris Nauroth via jghoman)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2d2c180/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/SWebHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/SWebHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/SWebHdfs.java
index cd36393..04b5880 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/SWebHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/SWebHdfs.java
@@ -46,6 +46,19 @@ public class SWebHdfs extends DelegateToFileSystem {
    */
   SWebHdfs(URI theUri, Configuration conf)
       throws IOException, URISyntaxException {
-    super(theUri, new SWebHdfsFileSystem(), conf, SCHEME, false);
+    super(theUri, createSWebHdfsFileSystem(conf), conf, SCHEME, false);
+  }
+
+  /**
+   * Returns a new {@link SWebHdfsFileSystem}, with the given configuration.
+   *
+   * @param conf configuration
+   * @return new SWebHdfsFileSystem
+   */
+  private static SWebHdfsFileSystem createSWebHdfsFileSystem(
+      Configuration conf) {
+    SWebHdfsFileSystem fs = new SWebHdfsFileSystem();
+    fs.setConf(conf);
+    return fs;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2d2c180/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/WebHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/WebHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/WebHdfs.java
index dc4f6d5..d54a6b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/WebHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/WebHdfs.java
@@ -46,6 +46,18 @@ public class WebHdfs extends DelegateToFileSystem {
    */
   WebHdfs(URI theUri, Configuration conf)
       throws IOException, URISyntaxException {
-    super(theUri, new WebHdfsFileSystem(), conf, SCHEME, false);
+    super(theUri, createWebHdfsFileSystem(conf), conf, SCHEME, false);
+  }
+
+  /**
+   * Returns a new {@link WebHdfsFileSystem}, with the given configuration.
+   *
+   * @param conf configuration
+   * @return new WebHdfsFileSystem
+   */
+  private static WebHdfsFileSystem createWebHdfsFileSystem(Configuration conf) {
+    WebHdfsFileSystem fs = new WebHdfsFileSystem();
+    fs.setConf(conf);
+    return fs;
   }
 }


[15/50] [abbrv] hadoop git commit: HADOOP-10365. BufferedOutputStream in FileUtil#unpackEntries() should be closed in finally block. Contributed by Kiran Kumar M R and Sanghyun Yun.

Posted by wa...@apache.org.
HADOOP-10365. BufferedOutputStream in FileUtil#unpackEntries() should be closed in finally block. Contributed by Kiran Kumar M R and Sanghyun Yun.


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

Branch: refs/heads/YARN-1197
Commit: dd149adeace8727864371c5a1484c6534f8b450b
Parents: 2e251a7
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Sep 2 02:01:51 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Sep 2 02:01:51 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt       |  3 +++
 .../src/main/java/org/apache/hadoop/fs/FileUtil.java  | 14 +++++++-------
 2 files changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd149ade/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4eef964..70252d6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1123,6 +1123,9 @@ Release 2.7.2 - UNRELEASED
     HADOOP-12359. hadoop fs -getmerge doc is wrong.
     (Jagadesh Kiran N via aajisaka)
 
+    HADOOP-10365. BufferedOutputStream in FileUtil#unpackEntries() should be
+    closed in finally block. (Kiran Kumar M R and Sanghyun Yun via ozawa)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd149ade/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 8abb4eb..3c0e90d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -742,15 +742,15 @@ public class FileUtil {
 
     int count;
     byte data[] = new byte[2048];
-    BufferedOutputStream outputStream = new BufferedOutputStream(
-        new FileOutputStream(outputFile));
+    try (BufferedOutputStream outputStream = new BufferedOutputStream(
+        new FileOutputStream(outputFile));) {
 
-    while ((count = tis.read(data)) != -1) {
-      outputStream.write(data, 0, count);
-    }
+      while ((count = tis.read(data)) != -1) {
+        outputStream.write(data, 0, count);
+      }
 
-    outputStream.flush();
-    outputStream.close();
+      outputStream.flush();
+    }
   }
 
   /**


[27/50] [abbrv] hadoop git commit: YARN-4101. RM should print alert messages if Zookeeper and Resourcemanager gets connection issue. Contributed by Xuan Gong

Posted by wa...@apache.org.
YARN-4101. RM should print alert messages if Zookeeper and Resourcemanager gets connection issue. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-1197
Commit: 09c64ba1ba8be7a2ac31f4e42efb8c99b682399f
Parents: d31a41c
Author: Jian He <ji...@apache.org>
Authored: Wed Sep 2 17:45:23 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Sep 2 17:45:23 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/ha/ActiveStandbyElector.java     | 4 ++++
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../hadoop/yarn/server/resourcemanager/AdminService.java    | 9 +++++++++
 .../yarn/server/resourcemanager/EmbeddedElectorService.java | 4 ++++
 .../yarn/server/resourcemanager/webapp/AboutBlock.java      | 2 ++
 .../hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java | 5 +++++
 .../yarn/server/resourcemanager/webapp/RMWebAppFilter.java  | 5 ++++-
 .../yarn/server/resourcemanager/webapp/dao/ClusterInfo.java | 6 ++++++
 .../server/resourcemanager/webapp/TestRMWebServices.java    | 8 ++++++--
 9 files changed, 43 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
index e458181..fcbcfdf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
@@ -1141,4 +1141,8 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
       ((appData == null) ? "null" : StringUtils.byteToHexString(appData)) + 
       " cb=" + appClient;
   }
+
+  public String getHAZookeeperConnectionState() {
+    return this.zkConnectionState.name();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 13fe9b0..5e0c43d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -871,6 +871,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3893. Both RM in active state when Admin#transitionToActive failure 
     from refeshAll() (Bibin A Chundatt via rohithsharmaks)
 
+    YARN-4101. RM should print alert messages if Zookeeper and Resourcemanager
+    gets connection issue. (Xuan Gong via jianhe)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/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 d96ed8c..a95e22c 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
@@ -782,4 +782,13 @@ public class AdminService extends CompositeService implements
       throw logAndWrapException(e, user.getShortUserName(), argName, msg);
     }
   }
+
+  public String getHAZookeeperConnectionState() {
+    if (!rmContext.isHAEnabled()) {
+      return "ResourceManager HA is not enabled.";
+    } else if (!autoFailoverEnabled) {
+      return "Auto Failover is not enabled.";
+    }
+    return this.embeddedElector.getHAZookeeperConnectionState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.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/EmbeddedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
index c7b7768..73bdca0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/EmbeddedElectorService.java
@@ -205,4 +205,8 @@ public class EmbeddedElectorService extends AbstractService
     elector.quitElection(false);
     elector.joinElection(localActiveNodeInfo);
   }
+
+  public String getHAZookeeperConnectionState() {
+    return elector.getHAZookeeperConnectionState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/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 ea5c48a..4225afd 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
@@ -44,6 +44,8 @@ public class AboutBlock extends HtmlBlock {
       _("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() +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index a86ed4f..91c3d72 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -117,4 +117,9 @@ public class RMWebApp extends WebApp implements YarnWebParams {
     }
     return path;
   }
+
+  public String getHAZookeeperConnectionState() {
+    return rm.getRMContext().getRMAdminService()
+      .getHAZookeeperConnectionState();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.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/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
index a8f793a..9f35b6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
@@ -58,6 +58,7 @@ public class RMWebAppFilter extends GuiceContainer {
   private String path;
   private static final int BASIC_SLEEP_TIME = 5;
   private static final int MAX_SLEEP_TIME = 5 * 60;
+  private static final Random randnum = new Random();
 
   @Inject
   public RMWebAppFilter(Injector injector, Configuration conf) {
@@ -126,6 +127,8 @@ public class RMWebAppFilter extends GuiceContainer {
         String redirectMsg =
             doRetry ? "Can not find any active RM. Will retry in next " + next
                 + " seconds." : "There is no active RM right now.";
+        redirectMsg += "\nHA Zookeeper Connection State: "
+            + rmWebApp.getHAZookeeperConnectionState();
         PrintWriter out = response.getWriter();
         out.println(redirectMsg);
         if (doRetry) {
@@ -172,6 +175,6 @@ public class RMWebAppFilter extends GuiceContainer {
 
   private static int calculateExponentialTime(int retries) {
     long baseTime = BASIC_SLEEP_TIME * (1L << retries);
-    return (int) (baseTime * ((new Random()).nextDouble() + 0.5));
+    return (int) (baseTime * (randnum.nextDouble() + 0.5));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java
index b529f21..512a5c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java
@@ -43,6 +43,7 @@ public class ClusterInfo {
   protected String hadoopVersion;
   protected String hadoopBuildVersion;
   protected String hadoopVersionBuiltOn;
+  protected String haZooKeeperConnectionState;
 
   public ClusterInfo() {
   } // JAXB needs this
@@ -62,6 +63,8 @@ public class ClusterInfo {
     this.hadoopVersion = VersionInfo.getVersion();
     this.hadoopBuildVersion = VersionInfo.getBuildVersion();
     this.hadoopVersionBuiltOn = VersionInfo.getDate();
+    this.haZooKeeperConnectionState =
+        rm.getRMContext().getRMAdminService().getHAZookeeperConnectionState();
   }
 
   public String getState() {
@@ -108,4 +111,7 @@ public class ClusterInfo {
     return this.startedOn;
   }
 
+  public String getHAZookeeperConnectionState() {
+    return this.haZooKeeperConnectionState;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c64ba1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.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/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
index 752e99b..530c06f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java
@@ -285,6 +285,8 @@ public class TestRMWebServices extends JerseyTestBase {
           WebServicesTestUtils.getXmlLong(element, "startedOn"),
           WebServicesTestUtils.getXmlString(element, "state"),
           WebServicesTestUtils.getXmlString(element, "haState"),
+          WebServicesTestUtils.getXmlString(
+              element, "haZooKeeperConnectionState"),
           WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"),
           WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"),
           WebServicesTestUtils.getXmlString(element, "hadoopVersion"),
@@ -300,9 +302,10 @@ public class TestRMWebServices extends JerseyTestBase {
       Exception {
     assertEquals("incorrect number of elements", 1, json.length());
     JSONObject info = json.getJSONObject("clusterInfo");
-    assertEquals("incorrect number of elements", 11, info.length());
+    assertEquals("incorrect number of elements", 12, info.length());
     verifyClusterGeneric(info.getLong("id"), info.getLong("startedOn"),
         info.getString("state"), info.getString("haState"),
+        info.getString("haZooKeeperConnectionState"),
         info.getString("hadoopVersionBuiltOn"),
         info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"),
         info.getString("resourceManagerVersionBuiltOn"),
@@ -312,7 +315,8 @@ public class TestRMWebServices extends JerseyTestBase {
   }
 
   public void verifyClusterGeneric(long clusterid, long startedon,
-      String state, String haState, String hadoopVersionBuiltOn,
+      String state, String haState, String haZooKeeperConnectionState,
+      String hadoopVersionBuiltOn,
       String hadoopBuildVersion, String hadoopVersion,
       String resourceManagerVersionBuiltOn, String resourceManagerBuildVersion,
       String resourceManagerVersion) {


[39/50] [abbrv] hadoop git commit: HDFS-9021. Use a yellow elephant rather than a blue one in diagram.

Posted by wa...@apache.org.
HDFS-9021. Use a yellow elephant rather than a blue one in diagram.


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

Branch: refs/heads/YARN-1197
Commit: c83d13c64993c3a7f0f35142cddac19e1074976e
Parents: ed78b14
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 3 16:27:20 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 3 16:27:20 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 ++
 .../site/resources/images/LazyPersistWrites.png | Bin 107161 -> 90224 bytes
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c83d13c6/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ce3fbb4..b9b89aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -886,6 +886,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-9002. Move o.a.h.hdfs.net/*Peer classes to hdfs-client.
     (Mingliang Liu via wheat9)
 
+    HDFS-9021. Use a yellow elephant rather than a blue one in diagram. (wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c83d13c6/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
index b2bde93..a21e866 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png and b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png differ


[26/50] [abbrv] hadoop git commit: HDFS-8890. Allow admin to specify which blockpools the balancer should run on. (Chris Trezzo via mingma)

Posted by wa...@apache.org.
HDFS-8890. Allow admin to specify which blockpools the balancer should run on. (Chris Trezzo via mingma)


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

Branch: refs/heads/YARN-1197
Commit: d31a41c35927f02f2fb40d19380b5df4bb2b6d57
Parents: de928d5
Author: Ming Ma <mi...@apache.org>
Authored: Wed Sep 2 15:55:42 2015 -0700
Committer: Ming Ma <mi...@apache.org>
Committed: Wed Sep 2 15:55:42 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/server/balancer/Balancer.java   |  82 ++++++---
 .../src/site/markdown/HDFSCommands.md           |   2 +
 .../hdfs/server/balancer/TestBalancer.java      |  43 ++++-
 .../TestBalancerWithMultipleNameNodes.java      | 179 ++++++++++++++++---
 5 files changed, 253 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d31a41c3/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7a685f5..e68c011 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -880,6 +880,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-328. Improve fs -setrep error message for invalid replication factors.
     (Daniel Templeton via wang)
 
+    HDFS-8890. Allow admin to specify which blockpools the balancer should run
+    on. (Chris Trezzo via mingma)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d31a41c3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index fe6e4c3..259b280 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -179,6 +179,8 @@ public class Balancer {
       + "\tExcludes the specified datanodes."
       + "\n\t[-include [-f <hosts-file> | <comma-separated list of hosts>]]"
       + "\tIncludes only the specified datanodes."
+      + "\n\t[-blockpools <comma-separated list of blockpool ids>]"
+      + "\tThe balancer will only run on blockpools included in this list."
       + "\n\t[-idleiterations <idleiterations>]"
       + "\tNumber of consecutive idle iterations (-1 for Infinite) before "
       + "exit."
@@ -652,22 +654,27 @@ public class Balancer {
         done = true;
         Collections.shuffle(connectors);
         for(NameNodeConnector nnc : connectors) {
-          final Balancer b = new Balancer(nnc, p, conf);
-          final Result r = b.runOneIteration();
-          r.print(iteration, System.out);
-
-          // clean all lists
-          b.resetData(conf);
-          if (r.exitStatus == ExitStatus.IN_PROGRESS) {
-            done = false;
-          } else if (r.exitStatus != ExitStatus.SUCCESS) {
-            //must be an error statue, return.
-            return r.exitStatus.getExitCode();
-          }
-        }
+          if (p.blockpools.size() == 0
+              || p.blockpools.contains(nnc.getBlockpoolID())) {
+            final Balancer b = new Balancer(nnc, p, conf);
+            final Result r = b.runOneIteration();
+            r.print(iteration, System.out);
+
+            // clean all lists
+            b.resetData(conf);
+            if (r.exitStatus == ExitStatus.IN_PROGRESS) {
+              done = false;
+            } else if (r.exitStatus != ExitStatus.SUCCESS) {
+              // must be an error statue, return.
+              return r.exitStatus.getExitCode();
+            }
 
-        if (!done) {
-          Thread.sleep(sleeptime);
+            if (!done) {
+              Thread.sleep(sleeptime);
+            }
+          } else {
+            LOG.info("Skipping blockpool " + nnc.getBlockpoolID());
+          }
         }
       }
     } finally {
@@ -699,12 +706,12 @@ public class Balancer {
   }
 
   static class Parameters {
-    static final Parameters DEFAULT = new Parameters(
-        BalancingPolicy.Node.INSTANCE, 10.0,
-        NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
-        Collections.<String>emptySet(), Collections.<String>emptySet(),
-        Collections.<String>emptySet(),
-        false);
+    static final Parameters DEFAULT =
+        new Parameters(BalancingPolicy.Node.INSTANCE, 10.0,
+            NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
+            Collections.<String> emptySet(), Collections.<String> emptySet(),
+            Collections.<String> emptySet(), Collections.<String> emptySet(),
+            false);
 
     final BalancingPolicy policy;
     final double threshold;
@@ -718,19 +725,25 @@ public class Balancer {
      */
     final Set<String> sourceNodes;
     /**
+     * A set of block pools to run the balancer on.
+     */
+    final Set<String> blockpools;
+    /**
      * Whether to run the balancer during upgrade.
      */
     final boolean runDuringUpgrade;
 
     Parameters(BalancingPolicy policy, double threshold, int maxIdleIteration,
         Set<String> excludedNodes, Set<String> includedNodes,
-        Set<String> sourceNodes, boolean runDuringUpgrade) {
+        Set<String> sourceNodes, Set<String> blockpools,
+        boolean runDuringUpgrade) {
       this.policy = policy;
       this.threshold = threshold;
       this.maxIdleIteration = maxIdleIteration;
       this.excludedNodes = excludedNodes;
       this.includedNodes = includedNodes;
       this.sourceNodes = sourceNodes;
+      this.blockpools = blockpools;
       this.runDuringUpgrade = runDuringUpgrade;
     }
 
@@ -742,10 +755,11 @@ public class Balancer {
               + " #excluded nodes = %s,"
               + " #included nodes = %s,"
               + " #source nodes = %s,"
+              + " #blockpools = %s,"
               + " run during upgrade = %s]",
-          Balancer.class.getSimpleName(), getClass().getSimpleName(),
-          policy, threshold, maxIdleIteration,
-          excludedNodes.size(), includedNodes.size(), sourceNodes.size(),
+          Balancer.class.getSimpleName(), getClass().getSimpleName(), policy,
+          threshold, maxIdleIteration, excludedNodes.size(),
+          includedNodes.size(), sourceNodes.size(), blockpools.size(),
           runDuringUpgrade);
     }
   }
@@ -789,6 +803,7 @@ public class Balancer {
       Set<String> excludedNodes = Parameters.DEFAULT.excludedNodes;
       Set<String> includedNodes = Parameters.DEFAULT.includedNodes;
       Set<String> sourceNodes = Parameters.DEFAULT.sourceNodes;
+      Set<String> blockpools = Parameters.DEFAULT.blockpools;
       boolean runDuringUpgrade = Parameters.DEFAULT.runDuringUpgrade;
 
       if (args != null) {
@@ -828,6 +843,14 @@ public class Balancer {
             } else if ("-source".equalsIgnoreCase(args[i])) {
               sourceNodes = new HashSet<>();
               i = processHostList(args, i, "source", sourceNodes);
+            } else if ("-blockpools".equalsIgnoreCase(args[i])) {
+              checkArgument(
+                  ++i < args.length,
+                  "blockpools value is missing: args = "
+                      + Arrays.toString(args));
+              blockpools = parseBlockPoolList(args[i]);
+              LOG.info("Balancer will run on the following blockpools: "
+                  + blockpools.toString());
             } else if ("-idleiterations".equalsIgnoreCase(args[i])) {
               checkArgument(++i < args.length,
                   "idleiterations value is missing: args = " + Arrays
@@ -853,8 +876,8 @@ public class Balancer {
         }
       }
       
-      return new Parameters(policy, threshold, maxIdleIteration,
-          excludedNodes, includedNodes, sourceNodes, runDuringUpgrade);
+      return new Parameters(policy, threshold, maxIdleIteration, excludedNodes,
+          includedNodes, sourceNodes, blockpools, runDuringUpgrade);
     }
 
     private static int processHostList(String[] args, int i, String type,
@@ -881,6 +904,11 @@ public class Balancer {
       return i;
     }
 
+    private static Set<String> parseBlockPoolList(String string) {
+      String[] addrs = StringUtils.getTrimmedStrings(string);
+      return new HashSet<String>(Arrays.asList(addrs));
+    }
+
     private static void printUsage(PrintStream out) {
       out.println(USAGE + "\n");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d31a41c3/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 1c2026c..8bbcbb8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -266,6 +266,7 @@ Usage:
               [-policy <policy>]
               [-exclude [-f <hosts-file> | <comma-separated list of hosts>]]
               [-include [-f <hosts-file> | <comma-separated list of hosts>]]
+              [-blockpools <comma-separated list of blockpool ids>]
               [-idleiterations <idleiterations>]
 
 | COMMAND\_OPTION | Description |
@@ -274,6 +275,7 @@ Usage:
 | `-threshold` \<threshold\> | Percentage of disk capacity. This overwrites the default threshold. |
 | `-exclude -f` \<hosts-file\> \| \<comma-separated list of hosts\> | Excludes the specified datanodes from being balanced by the balancer. |
 | `-include -f` \<hosts-file\> \| \<comma-separated list of hosts\> | Includes only the specified datanodes to be balanced by the balancer. |
+| `-blockpools` \<comma-separated list of blockpool ids\> | The balancer will only run on blockpools included in this list. |
 | `-idleiterations` \<iterations\> | Maximum number of idle iterations before exit. This overwrites the default idleiterations(5). |
 
 Runs a cluster balancing utility. An administrator can simply press Ctrl-C to stop the rebalancing process. See [Balancer](./HdfsUserGuide.html#Balancer) for more details.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d31a41c3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index c1ed758..b0223d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -644,7 +644,7 @@ public class TestBalancer {
             Balancer.Parameters.DEFAULT.maxIdleIteration,
             nodes.getNodesToBeExcluded(), nodes.getNodesToBeIncluded(),
             Balancer.Parameters.DEFAULT.sourceNodes,
-            false);
+            Balancer.Parameters.DEFAULT.blockpools, false);
       }
 
       int expectedExcludedNodes = 0;
@@ -885,7 +885,7 @@ public class TestBalancer {
           Balancer.Parameters.DEFAULT.maxIdleIteration,
           datanodes, Balancer.Parameters.DEFAULT.includedNodes,
           Balancer.Parameters.DEFAULT.sourceNodes,
-          false);
+          Balancer.Parameters.DEFAULT.blockpools, false);
       final int r = Balancer.run(namenodes, p, conf);
       assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
     } finally {
@@ -1067,6 +1067,34 @@ public class TestBalancer {
     } catch (IllegalArgumentException e) {
 
     }
+
+    parameters = new String[] { "-blockpools" };
+    try {
+      Balancer.Cli.parse(parameters);
+      fail("IllegalArgumentException is expected when a value "
+          + "is not specified for the blockpool flag");
+    } catch (IllegalArgumentException e) {
+
+    }
+  }
+
+  @Test
+  public void testBalancerCliParseBlockpools() {
+    String[] parameters = new String[] { "-blockpools", "bp-1,bp-2,bp-3" };
+    Balancer.Parameters p = Balancer.Cli.parse(parameters);
+    assertEquals(3, p.blockpools.size());
+
+    parameters = new String[] { "-blockpools", "bp-1" };
+    p = Balancer.Cli.parse(parameters);
+    assertEquals(1, p.blockpools.size());
+
+    parameters = new String[] { "-blockpools", "bp-1,,bp-2" };
+    p = Balancer.Cli.parse(parameters);
+    assertEquals(3, p.blockpools.size());
+
+    parameters = new String[] { "-blockpools", "bp-1," };
+    p = Balancer.Cli.parse(parameters);
+    assertEquals(1, p.blockpools.size());
   }
 
   /**
@@ -1385,7 +1413,7 @@ public class TestBalancer {
               Parameters.DEFAULT.excludedNodes,
               Parameters.DEFAULT.includedNodes,
               Parameters.DEFAULT.sourceNodes,
-              true);
+              Balancer.Parameters.DEFAULT.blockpools, true);
       assertEquals(ExitStatus.SUCCESS.getExitCode(),
           Balancer.run(namenodes, runDuringUpgrade, conf));
 
@@ -1588,7 +1616,8 @@ public class TestBalancer {
             BalancingPolicy.Node.INSTANCE, 1,
             NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
             Collections.<String> emptySet(), Collections.<String> emptySet(),
-            Collections.<String> emptySet(), false);
+            Collections.<String> emptySet(),
+            Balancer.Parameters.DEFAULT.blockpools, false);
 
         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
         final int r = Balancer.run(namenodes, p, conf);
@@ -1607,7 +1636,7 @@ public class TestBalancer {
           BalancingPolicy.Node.INSTANCE, 1,
           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
           Collections.<String> emptySet(), Collections.<String> emptySet(),
-          sourceNodes, false);
+          sourceNodes, Balancer.Parameters.DEFAULT.blockpools, false);
 
         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
         final int r = Balancer.run(namenodes, p, conf);
@@ -1622,7 +1651,7 @@ public class TestBalancer {
           BalancingPolicy.Node.INSTANCE, 1,
           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
           Collections.<String> emptySet(), Collections.<String> emptySet(),
-          sourceNodes, false);
+          sourceNodes, Balancer.Parameters.DEFAULT.blockpools, false);
 
         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
         final int r = Balancer.run(namenodes, p, conf);
@@ -1639,7 +1668,7 @@ public class TestBalancer {
           BalancingPolicy.Node.INSTANCE, 1,
           NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
           Collections.<String> emptySet(), Collections.<String> emptySet(),
-          sourceNodes, false);
+          sourceNodes, Balancer.Parameters.DEFAULT.blockpools, false);
 
         conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
         final int r = Balancer.run(namenodes, p, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d31a41c3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
index f51757c..b07ad89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
@@ -21,8 +21,13 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -42,6 +47,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
@@ -60,6 +67,7 @@ public class TestBalancerWithMultipleNameNodes {
   private static final long CAPACITY = 500L;
   private static final String RACK0 = "/rack0";
   private static final String RACK1 = "/rack1";
+  private static final String RACK2 = "/rack2";
 
   private static final String FILE_NAME = "/tmp.txt";
   private static final Path FILE_PATH = new Path(FILE_NAME);
@@ -76,16 +84,20 @@ public class TestBalancerWithMultipleNameNodes {
     final MiniDFSCluster cluster;
     final ClientProtocol[] clients;
     final short replication;
-    
+    final Balancer.Parameters parameters;
+
     Suite(MiniDFSCluster cluster, final int nNameNodes, final int nDataNodes,
-        Configuration conf) throws IOException {
+        Balancer.Parameters parameters, Configuration conf) throws IOException {
       this.conf = conf;
       this.cluster = cluster;
       clients = new ClientProtocol[nNameNodes];
       for(int i = 0; i < nNameNodes; i++) {
         clients[i] = cluster.getNameNode(i).getRpcServer();
       }
-      replication = (short)Math.max(1, nDataNodes - 1);
+      // hard coding replication factor to 1 so logical and raw HDFS size are
+      // equal
+      replication = 1;
+      this.parameters = parameters;
     }
   }
 
@@ -104,11 +116,9 @@ public class TestBalancerWithMultipleNameNodes {
       ) throws IOException, InterruptedException, TimeoutException {
     final ExtendedBlock[][] blocks = new ExtendedBlock[s.clients.length][];
     for(int n = 0; n < s.clients.length; n++) {
-      final long fileLen = size/s.replication;
-      createFile(s, n, fileLen);
-
-      final List<LocatedBlock> locatedBlocks = s.clients[n].getBlockLocations(
-          FILE_NAME, 0, fileLen).getLocatedBlocks();
+      createFile(s, n, size);
+      final List<LocatedBlock> locatedBlocks =
+          s.clients[n].getBlockLocations(FILE_NAME, 0, size).getLocatedBlocks();
 
       final int numOfBlocks = locatedBlocks.size();
       blocks[n] = new ExtendedBlock[numOfBlocks];
@@ -151,9 +161,14 @@ public class TestBalancerWithMultipleNameNodes {
     wait(s.clients, totalUsed, totalCapacity);
     LOG.info("BALANCER 1");
 
+    // get storage reports for relevant blockpools so that we can compare
+    // blockpool usages after balancer has run
+    Map<Integer, DatanodeStorageReport[]> preBalancerPoolUsages =
+        getStorageReports(s);
+
     // start rebalancing
     final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(s.conf);
-    final int r = Balancer.run(namenodes, Balancer.Parameters.DEFAULT, s.conf);
+    final int r = Balancer.run(namenodes, s.parameters, s.conf);
     Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
 
     LOG.info("BALANCER 2");
@@ -189,7 +204,7 @@ public class TestBalancerWithMultipleNameNodes {
       balanced = true;
       for(int d = 0; d < used.length; d++) {
         final double p = used[d]*100.0/cap[d];
-        balanced = p <= avg + Balancer.Parameters.DEFAULT.threshold;
+        balanced = p <= avg + s.parameters.threshold;
         if (!balanced) {
           if (i % 100 == 0) {
             LOG.warn("datanodes " + d + " is not yet balanced: "
@@ -203,6 +218,89 @@ public class TestBalancerWithMultipleNameNodes {
       }
     }
     LOG.info("BALANCER 6");
+    // cluster is balanced, verify that only selected blockpools were touched
+    Map<Integer, DatanodeStorageReport[]> postBalancerPoolUsages =
+        getStorageReports(s);
+    Assert.assertEquals(preBalancerPoolUsages.size(),
+        postBalancerPoolUsages.size());
+    for (Map.Entry<Integer, DatanodeStorageReport[]> entry
+        : preBalancerPoolUsages.entrySet()) {
+      compareTotalPoolUsage(entry.getValue(),
+          postBalancerPoolUsages.get(entry.getKey()));
+    }
+  }
+
+  /**
+   * Compare the total blockpool usage on each datanode to ensure that nothing
+   * was balanced.
+   *
+   * @param preReports storage reports from pre balancer run
+   * @param postReports storage reports from post balancer run
+   */
+  private static void compareTotalPoolUsage(DatanodeStorageReport[] preReports,
+      DatanodeStorageReport[] postReports) {
+    Assert.assertNotNull(preReports);
+    Assert.assertNotNull(postReports);
+    Assert.assertEquals(preReports.length, postReports.length);
+    for (DatanodeStorageReport preReport : preReports) {
+      String dnUuid = preReport.getDatanodeInfo().getDatanodeUuid();
+      for(DatanodeStorageReport postReport : postReports) {
+        if(postReport.getDatanodeInfo().getDatanodeUuid().equals(dnUuid)) {
+          Assert.assertEquals(getTotalPoolUsage(preReport),
+              getTotalPoolUsage(postReport));
+          LOG.info("Comparision of datanode pool usage pre/post balancer run. "
+              + "PrePoolUsage: " + getTotalPoolUsage(preReport)
+              + ", PostPoolUsage: " + getTotalPoolUsage(postReport));
+          break;
+        }
+      }
+    }
+  }
+
+  private static long getTotalPoolUsage(DatanodeStorageReport report) {
+    long usage = 0L;
+    for (StorageReport sr : report.getStorageReports()) {
+      usage += sr.getBlockPoolUsed();
+    }
+    return usage;
+  }
+
+  /**
+   * Get the storage reports for all blockpools that were not specified by the
+   * balancer blockpool parameters. If none were specified then the parameter
+   * was not set and do not return any reports.
+   *
+   * @param s suite for the test
+   * @return a map of storage reports where the key is the blockpool index
+   * @throws IOException
+   */
+  private static Map<Integer,
+    DatanodeStorageReport[]> getStorageReports(Suite s) throws IOException {
+    Map<Integer, DatanodeStorageReport[]> reports =
+        new HashMap<Integer, DatanodeStorageReport[]>();
+    if (s.parameters.blockpools.size() == 0) {
+      // the blockpools parameter was not set, so we don't need to track any
+      // blockpools.
+      return Collections.emptyMap();
+    }
+    for (int i = 0; i < s.clients.length; i++) {
+      if (s.parameters.blockpools.contains(s.cluster.getNamesystem(i)
+          .getBlockPoolId())) {
+        // we want to ensure that blockpools not specified by the balancer
+        // parameters were left alone. Therefore, if the pool was specified,
+        // skip it. Note: this code assumes the clients in the suite are ordered
+        // the same way that they are indexed via cluster#getNamesystem(index).
+        continue;
+      } else {
+        LOG.info("Tracking usage of blockpool id: "
+            + s.cluster.getNamesystem(i).getBlockPoolId());
+        reports.put(i,
+            s.clients[i].getDatanodeStorageReport(DatanodeReportType.LIVE));
+      }
+    }
+    LOG.info("Tracking " + reports.size()
+        + " blockpool(s) for pre/post balancer usage.");
+    return reports;
   }
 
   private static void sleep(long ms) {
@@ -220,25 +318,31 @@ public class TestBalancerWithMultipleNameNodes {
   }
 
   /**
-   * First start a cluster and fill the cluster up to a certain size.
-   * Then redistribute blocks according the required distribution.
-   * Finally, balance the cluster.
-   * 
+   * First start a cluster and fill the cluster up to a certain size. Then
+   * redistribute blocks according the required distribution. Finally, balance
+   * the cluster.
+   *
    * @param nNameNodes Number of NameNodes
-   * @param distributionPerNN The distribution for each NameNode. 
+   * @param nNameNodesToBalance Number of NameNodes to run the balancer on
+   * @param distributionPerNN The distribution for each NameNode.
    * @param capacities Capacities of the datanodes
    * @param racks Rack names
    * @param conf Configuration
    */
   private void unevenDistribution(final int nNameNodes,
-      long distributionPerNN[], long capacities[], String[] racks,
-      Configuration conf) throws Exception {
+      final int nNameNodesToBalance, long distributionPerNN[],
+      long capacities[], String[] racks, Configuration conf) throws Exception {
     LOG.info("UNEVEN 0");
     final int nDataNodes = distributionPerNN.length;
     if (capacities.length != nDataNodes || racks.length != nDataNodes) {
       throw new IllegalArgumentException("Array length is not the same");
     }
 
+    if (nNameNodesToBalance > nNameNodes) {
+      throw new IllegalArgumentException("Number of namenodes to balance is "
+          + "greater than the number of namenodes.");
+    }
+
     // calculate total space that need to be filled
     final long usedSpacePerNN = TestBalancer.sum(distributionPerNN);
 
@@ -248,7 +352,7 @@ public class TestBalancerWithMultipleNameNodes {
       LOG.info("UNEVEN 1");
       final MiniDFSCluster cluster = new MiniDFSCluster
           .Builder(new Configuration(conf))
-          .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+              .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
           .numDataNodes(nDataNodes)
           .racks(racks)
           .simulatedCapacities(capacities)
@@ -258,7 +362,7 @@ public class TestBalancerWithMultipleNameNodes {
         cluster.waitActive();
         DFSTestUtil.setFederatedConfiguration(cluster, conf);
         LOG.info("UNEVEN 3");
-        final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
+        final Suite s = new Suite(cluster, nNameNodes, nDataNodes, null, conf);
         blocks = generateBlocks(s, usedSpacePerNN);
         LOG.info("UNEVEN 4");
       } finally {
@@ -280,7 +384,20 @@ public class TestBalancerWithMultipleNameNodes {
       try {
         cluster.waitActive();
         LOG.info("UNEVEN 12");
-        final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
+        Set<String> blockpools = new HashSet<String>();
+        for (int i = 0; i < nNameNodesToBalance; i++) {
+          blockpools.add(cluster.getNamesystem(i).getBlockPoolId());
+        }
+        Balancer.Parameters params =
+            new Balancer.Parameters(Balancer.Parameters.DEFAULT.policy,
+                Balancer.Parameters.DEFAULT.threshold,
+                Balancer.Parameters.DEFAULT.maxIdleIteration,
+                Balancer.Parameters.DEFAULT.excludedNodes,
+                Balancer.Parameters.DEFAULT.includedNodes,
+                Balancer.Parameters.DEFAULT.sourceNodes, blockpools,
+                Balancer.Parameters.DEFAULT.runDuringUpgrade);
+        final Suite s =
+            new Suite(cluster, nNameNodes, nDataNodes, params, conf);
         for(int n = 0; n < nNameNodes; n++) {
           // redistribute blocks
           final Block[][] blocksDN = TestBalancer.distributeBlocks(
@@ -336,7 +453,9 @@ public class TestBalancerWithMultipleNameNodes {
     try {
       cluster.waitActive();
       LOG.info("RUN_TEST 1");
-      final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
+      final Suite s =
+          new Suite(cluster, nNameNodes, nDataNodes,
+              Balancer.Parameters.DEFAULT, conf);
       long totalCapacity = TestBalancer.sum(capacities);
 
       LOG.info("RUN_TEST 2");
@@ -378,10 +497,26 @@ public class TestBalancerWithMultipleNameNodes {
   @Test
   public void testUnevenDistribution() throws Exception {
     final Configuration conf = createConf();
-    unevenDistribution(2,
+    unevenDistribution(2, 2,
         new long[] {30*CAPACITY/100, 5*CAPACITY/100},
         new long[]{CAPACITY, CAPACITY},
         new String[] {RACK0, RACK1},
         conf);
   }
+
+  @Test
+  public void testBalancing1OutOf2Blockpools() throws Exception {
+    final Configuration conf = createConf();
+    unevenDistribution(2, 1, new long[] { 30 * CAPACITY / 100,
+        5 * CAPACITY / 100 }, new long[] { CAPACITY, CAPACITY }, new String[] {
+        RACK0, RACK1 }, conf);
+  }
+
+  @Test
+  public void testBalancing2OutOf3Blockpools() throws Exception {
+    final Configuration conf = createConf();
+    unevenDistribution(3, 2, new long[] { 30 * CAPACITY / 100,
+        5 * CAPACITY / 100, 10 * CAPACITY / 100 }, new long[] { CAPACITY,
+        CAPACITY, CAPACITY }, new String[] { RACK0, RACK1, RACK2 }, conf);
+  }
 }


[32/50] [abbrv] hadoop git commit: HADOOP-12213. Interrupted exception can occur when Client#stop is called. Contributed by Kuhu Shukla.

Posted by wa...@apache.org.
HADOOP-12213. Interrupted exception can occur when Client#stop is called. Contributed by Kuhu Shukla.


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

Branch: refs/heads/YARN-1197
Commit: 0ebc658105336cfe3e1a248b411de60f1d380928
Parents: 355eaaa
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu Sep 3 23:32:42 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Thu Sep 3 23:32:42 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../main/java/org/apache/hadoop/ipc/Client.java |  9 +++--
 .../java/org/apache/hadoop/ipc/TestIPC.java     | 38 +++++++++++++++++++-
 3 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ebc6581/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 20474be..512ca1b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1132,6 +1132,9 @@ Release 2.7.2 - UNRELEASED
     HADOOP-10365. BufferedOutputStream in FileUtil#unpackEntries() should be
     closed in finally block. (Kiran Kumar M R and Sanghyun Yun via ozawa)
 
+    HADOOP-12213. Interrupted exception can occur when Client#stop is called.
+    (Kuhu Shukla via ozawa)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ebc6581/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 6996a51..9087e5c 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
@@ -197,9 +197,10 @@ public class Client {
             clientExecutor.shutdownNow();
           }
         } catch (InterruptedException e) {
-          LOG.error("Interrupted while waiting for clientExecutor" +
-              "to stop", e);
+          LOG.warn("Interrupted while waiting for clientExecutor" +
+              " to stop");
           clientExecutor.shutdownNow();
+          Thread.currentThread().interrupt();
         }
         clientExecutor = null;
       }
@@ -256,6 +257,10 @@ public class Client {
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_TIMEOUT_KEY, timeout);
   }
 
+  @VisibleForTesting
+  public static final ExecutorService getClientExecutor() {
+    return Client.clientExcecutorFactory.clientExecutor;
+  }
   /**
    * Increment this client's reference count
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0ebc6581/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 08508ae..4e2e2f1 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
@@ -304,6 +304,8 @@ public class TestIPC {
       String causeText=cause.getMessage();
       assertTrue("Did not find " + causeText + " in " + message,
               message.contains(causeText));
+    } finally {
+      client.stop();
     }
   }
   
@@ -416,6 +418,7 @@ public class TestIPC {
       client.call(param, addr, null, null, 0, conf);
       
     } finally {
+      client.stop();
       server.stop();
     }
   }
@@ -531,6 +534,8 @@ public class TestIPC {
       fail("Expected an exception to have been thrown");
     } catch (IOException e) {
       assertTrue(e.getMessage().contains("Injected fault"));
+    } finally {
+      client.stop();
     }
   }
 
@@ -556,11 +561,11 @@ public class TestIPC {
     }).when(spyFactory).createSocket();
       
     Server server = new TestServer(1, true);
+    Client client = new Client(LongWritable.class, conf, spyFactory);
     server.start();
     try {
       // Call should fail due to injected exception.
       InetSocketAddress address = NetUtils.getConnectAddress(server);
-      Client client = new Client(LongWritable.class, conf, spyFactory);
       try {
         client.call(new LongWritable(RANDOM.nextLong()),
                 address, null, null, 0, conf);
@@ -577,6 +582,7 @@ public class TestIPC {
       client.call(new LongWritable(RANDOM.nextLong()),
           address, null, null, 0, conf);
     } finally {
+      client.stop();
       server.stop();
     }
   }
@@ -601,6 +607,7 @@ public class TestIPC {
     // set timeout to be bigger than 3*ping interval
     client.call(new LongWritable(RANDOM.nextLong()),
         addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
+    client.stop();
   }
 
   @Test(timeout=60000)
@@ -621,6 +628,7 @@ public class TestIPC {
     } catch (SocketTimeoutException e) {
       LOG.info("Get a SocketTimeoutException ", e);
     }
+    client.stop();
   }
   
   /**
@@ -851,6 +859,8 @@ public class TestIPC {
             } catch (IOException e) {
               LOG.error(e);
             } catch (InterruptedException e) {
+            } finally {
+              client.stop();
             }
           }
         });
@@ -952,6 +962,31 @@ public class TestIPC {
         endFds - startFds < 20);
   }
   
+  /**
+   * Check if Client is interrupted after handling
+   * InterruptedException during cleanup
+   */
+  @Test(timeout=30000)
+  public void testInterrupted() {
+    Client client = new Client(LongWritable.class, conf);
+    client.getClientExecutor().submit(new Runnable() {
+      public void run() {
+        while(true);
+      }
+    });
+    Thread.currentThread().interrupt();
+    client.stop();
+    try {
+      assertTrue(Thread.currentThread().isInterrupted());
+      LOG.info("Expected thread interrupt during client cleanup");
+    } catch (AssertionError e) {
+      LOG.error("The Client did not interrupt after handling an Interrupted Exception");
+      Assert.fail("The Client did not interrupt after handling an Interrupted Exception");
+    }
+    // Clear Thread interrupt
+    Thread.currentThread().interrupted();
+  }
+
   private long countOpenFileDescriptors() {
     return FD_DIR.list().length;
   }
@@ -1315,6 +1350,7 @@ public class TestIPC {
       Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))
           .createSocket();
     }
+    client.stop();
   }
   
   private void doIpcVersionTest(


[24/50] [abbrv] hadoop git commit: HDFS-328. Improve fs -setrep error message for invalid replication factors. Contributed by Daniel Templeton.

Posted by wa...@apache.org.
HDFS-328. Improve fs -setrep error message for invalid replication factors. Contributed by Daniel Templeton.


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

Branch: refs/heads/YARN-1197
Commit: afc88b396f06488c331564e0f6987013bb920d3e
Parents: c006c3a
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Sep 2 13:45:20 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Sep 2 13:46:00 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../server/blockmanagement/BlockManager.java    | 41 ++++++++------
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 56 ++++++++++++++++++++
 .../src/test/resources/testHDFSConf.xml         | 56 +++++++++++++++++++-
 4 files changed, 140 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc88b39/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0f2d713..78bbf26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -877,6 +877,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-2070. Add more unit tests for FsShell getmerge (Daniel Templeton via
     Colin P. McCabe)
 
+    HDFS-328. Improve fs -setrep error message for invalid replication factors.
+    (Daniel Templeton via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc88b39/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 1346ab3..08fbd4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -953,28 +953,39 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Check whether the replication parameter is within the range
-   * determined by system configuration.
+   * determined by system configuration and throw an exception if it's not.
+   *
+   * @param src the path to the target file
+   * @param replication the requested replication factor
+   * @param clientName the name of the client node making the request
+   * @throws java.io.IOException thrown if the requested replication factor
+   * is out of bounds
    */
    public void verifyReplication(String src,
                           short replication,
                           String clientName) throws IOException {
 
-    if (replication >= minReplication && replication <= maxReplication) {
-      //common case. avoid building 'text'
-      return;
-    }
-    
-    String text = "file " + src 
-      + ((clientName != null) ? " on client " + clientName : "")
-      + ".\n"
-      + "Requested replication " + replication;
+    if (replication < minReplication || replication > maxReplication) {
+      StringBuilder msg = new StringBuilder("Requested replication factor of ");
 
-    if (replication > maxReplication)
-      throw new IOException(text + " exceeds maximum " + maxReplication);
+      msg.append(replication);
 
-    if (replication < minReplication)
-      throw new IOException(text + " is less than the required minimum " +
-                            minReplication);
+      if (replication > maxReplication) {
+        msg.append(" exceeds maximum of ");
+        msg.append(maxReplication);
+      } else {
+        msg.append(" is less than the required minimum of ");
+        msg.append(minReplication);
+      }
+
+      msg.append(" for ").append(src);
+
+      if (clientName != null) {
+        msg.append(" from ").append(clientName);
+      }
+
+      throw new IOException(msg.toString());
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc88b39/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 1386124..dda2051 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
@@ -2412,7 +2412,63 @@ public class TestDFSShell {
     }
   }
 
+  /**
+   * Test -setrep with a replication factor that is too low.  We have to test
+   * this here because the mini-cluster used with testHDFSConf.xml uses a
+   * replication factor of 1 (for good reason).
+   */
+  @Test (timeout = 30000)
+  public void testSetrepLow() throws Exception {
+    Configuration conf = new Configuration();
+
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, 2);
+
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    MiniDFSCluster cluster = builder.numDataNodes(2).format(true).build();
+    FsShell shell = new FsShell(conf);
+
+    cluster.waitActive();
+
+    final String testdir = "/tmp/TestDFSShell-testSetrepLow";
+    final Path hdfsFile = new Path(testdir, "testFileForSetrepLow");
+    final PrintStream origOut = System.out;
+    final PrintStream origErr = System.err;
+
+    try {
+      final FileSystem fs = cluster.getFileSystem();
+
+      assertTrue("Unable to create test directory",
+          fs.mkdirs(new Path(testdir)));
 
+      fs.create(hdfsFile, true).close();
+
+      // Capture the command output so we can examine it
+      final ByteArrayOutputStream bao = new ByteArrayOutputStream();
+      final PrintStream capture = new PrintStream(bao);
+
+      System.setOut(capture);
+      System.setErr(capture);
+
+      final String[] argv = new String[] { "-setrep", "1", hdfsFile.toString() };
+
+      try {
+        assertEquals("Command did not return the expected exit code",
+            1, shell.run(argv));
+      } finally {
+        System.setOut(origOut);
+        System.setErr(origErr);
+      }
+
+      assertEquals("Error message is not the expected error message",
+          "setrep: Requested replication factor of 1 is less than "
+              + "the required minimum of 2 for /tmp/TestDFSShell-"
+              + "testSetrepLow/testFileForSetrepLow\n",
+          bao.toString());
+    } finally {
+      shell.close();
+      cluster.shutdown();
+    }
+  }
 
   // setrep for file and directory.
   @Test (timeout = 30000)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afc88b39/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 18c68ca..7c3cac9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -6471,7 +6471,61 @@
         </comparator>
       </comparators>
     </test>
-    
+
+    <test> <!-- TESTED -->
+      <description>setrep: invalid replication factor -- too high</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir0</command>
+        <command>-fs NAMENODE -touchz /dir0/file0</command>
+        <command>-fs NAMENODE -setrep 1025 /dir0/file0</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^setrep: Requested replication factor of 1025 exceeds maximum of [0-9]+ for /dir0/file0</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>setrep: invalid replication factor -- 0</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p dir0</command>
+        <command>-fs NAMENODE -touchz dir0/file0</command>
+        <command>-fs NAMENODE -setrep 0 dir0/file0</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^-setrep: replication must be >= 1</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!-- TESTED -->
+      <description>setrep: invalid replication factor -- NaN</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p dir0</command>
+        <command>-fs NAMENODE -touchz dir0/file0</command>
+        <command>-fs NAMENODE -setrep three dir0/file0</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -r /user</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^setrep: Illegal replication, a positive integer expected</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
     <!-- Tests for touchz-->
     <test> <!-- TESTED -->
       <description>touchz: touching file (absolute path) </description>


[16/50] [abbrv] hadoop git commit: HADOOP-12369. Point hadoop-project/pom.xml java.security.krb5.conf within target folder.

Posted by wa...@apache.org.
HADOOP-12369. Point hadoop-project/pom.xml java.security.krb5.conf within target folder.


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

Branch: refs/heads/YARN-1197
Commit: 0eb9b1932590ba6c60ac996d03269dffb4e095cc
Parents: dd149ad
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 1 10:57:32 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 1 10:57:32 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt              | 3 +++
 hadoop-project/pom.xml                                       | 2 +-
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0eb9b193/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 70252d6..e915290 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -762,6 +762,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12367. Move TestFileUtil's test resources to resources folder.
     (wang via yliu)
 
+    HADOOP-12369. Point hadoop-project/pom.xml java.security.krb5.conf
+    within target folder. (wang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0eb9b193/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 86102c6..9863475 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1132,7 +1132,7 @@
             <test.build.classes>${test.build.classes}</test.build.classes>
 
             <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
-            <java.security.krb5.conf>${basedir}/src/test/resources/krb5.conf</java.security.krb5.conf>
+            <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>
             <java.security.egd>${java.security.egd}</java.security.egd>
             <require.test.libhadoop>${require.test.libhadoop}</require.test.libhadoop>
           </systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0eb9b193/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
index 52994e7..635f693 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/pom.xml
@@ -198,7 +198,7 @@
           <test.build.classes>${test.build.classes}</test.build.classes>
 
           <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
-          <java.security.krb5.conf>${basedir}/src/test/resources/krb5.conf</java.security.krb5.conf>
+          <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>
           <java.security.egd>${java.security.egd}</java.security.egd>
           <require.test.libhadoop>${require.test.libhadoop}</require.test.libhadoop>
         </systemPropertyVariables>


[02/50] [abbrv] hadoop git commit: YARN-2945. Fixing the CHANGES.txt to have the right JIRA number.

Posted by wa...@apache.org.
YARN-2945. Fixing the CHANGES.txt to have the right JIRA number.


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

Branch: refs/heads/YARN-1197
Commit: cf831565e8344523e1bd0eaf686ed56a2b48b920
Parents: 837fb75
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Sun Aug 30 20:01:47 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Sun Aug 30 20:01:47 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cf831565/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0b733a4..4201b4f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1453,7 +1453,7 @@ Release 2.7.0 - 2015-04-20
     YARN-2914. [YARN-1492] Potential race condition in Singleton implementation of 
     SharedCacheUploaderMetrics, CleanerMetrics, ClientSCMMetrics. (Varun Saxena via kasha)
 
-    YARN-2964. FSLeafQueue#assignContainer - document the reason for using both write and
+    YARN-2945. FSLeafQueue#assignContainer - document the reason for using both write and
     read locks. (Tsuyoshi Ozawa via kasha)
 
     YARN-2944. InMemorySCMStore can not be instantiated with ReflectionUtils#newInstance.


[06/50] [abbrv] hadoop git commit: YARN-4092. Fixed UI redirection to print useful messages when both RMs are in standby mode. Contributed by Xuan Gong

Posted by wa...@apache.org.
YARN-4092. Fixed UI redirection to print useful messages when both RMs are in standby mode. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-1197
Commit: a3fd2ccc869dfc1f04d1cf0a8678d4d90a43a80f
Parents: 826ae1c
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 31 17:33:24 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Aug 31 17:33:24 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../hadoop/yarn/client/TestRMFailover.java      | 27 ++++++
 .../hadoop/yarn/webapp/YarnWebParams.java       |  1 +
 .../resourcemanager/webapp/RMWebAppFilter.java  | 90 +++++++++++++++++++-
 4 files changed, 117 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fd2ccc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 4201b4f..19c1082 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -422,6 +422,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1556. NPE getting application report with a null appId. (Weiwei Yang via 
     junping_du)
 
+    YARN-4092. Fixed UI redirection to print useful messages when both RMs are
+    in standby mode. (Xuan Gong via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fd2ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
index 0d03fd4..cbc220a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
@@ -27,6 +27,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
+
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.logging.Log;
@@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
 import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServer;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -265,6 +267,7 @@ public class TestRMFailover extends ClientBaseWithFixes {
     getAdminService(0).transitionToActive(req);
     String rm1Url = "http://0.0.0.0:18088";
     String rm2Url = "http://0.0.0.0:28088";
+
     String redirectURL = getRedirectURL(rm2Url);
     // if uri is null, RMWebAppFilter will append a slash at the trail of the redirection url
     assertEquals(redirectURL,rm1Url+"/");
@@ -304,6 +307,17 @@ public class TestRMFailover extends ClientBaseWithFixes {
 
     redirectURL = getRedirectURL(rm2Url + "/proxy/" + fakeAppId);
     assertNull(redirectURL);
+
+    // transit the active RM to standby
+    // Both of RMs are in standby mode
+    getAdminService(0).transitionToStandby(req);
+    // RM2 is expected to send the httpRequest to itself.
+    // The Header Field: Refresh is expected to be set.
+    redirectURL = getRefreshURL(rm2Url);
+    assertTrue(redirectURL != null
+        && redirectURL.contains(YarnWebParams.NEXT_REFRESH_INTERVAL)
+        && redirectURL.contains(rm2Url));
+
   }
 
   // set up http connection with the given url and get the redirection url from the response
@@ -323,4 +337,17 @@ public class TestRMFailover extends ClientBaseWithFixes {
     return redirectUrl;
   }
 
+  static String getRefreshURL(String url) {
+    String redirectUrl = null;
+    try {
+      HttpURLConnection conn = (HttpURLConnection) new URL(url).openConnection();
+      // do not automatically follow the redirection
+      // otherwise we get too many redirections exception
+      conn.setInstanceFollowRedirects(false);
+      redirectUrl = conn.getHeaderField("Refresh");
+    } catch (Exception e) {
+      // throw new RuntimeException(e);
+    }
+    return redirectUrl;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fd2ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
index 679e1d6..3792649 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
@@ -40,4 +40,5 @@ public interface YarnWebParams {
   String NODE_STATE = "node.state";
   String NODE_LABEL = "node.label";
   String WEB_UI_TYPE = "web.ui.type";
+  String NEXT_REFRESH_INTERVAL = "next.fresh.interval";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3fd2ccc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.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/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
index 500f17a..a8f793a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java
@@ -20,6 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Random;
 import java.util.Set;
 
 import javax.inject.Inject;
@@ -29,8 +33,11 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HtmlQuoting;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 import com.google.common.collect.Sets;
 import com.google.inject.Injector;
@@ -48,11 +55,26 @@ public class RMWebAppFilter extends GuiceContainer {
   // define a set of URIs which do not need to do redirection
   private static final Set<String> NON_REDIRECTED_URIS = Sets.newHashSet(
       "/conf", "/stacks", "/logLevel", "/logs");
+  private String path;
+  private static final int BASIC_SLEEP_TIME = 5;
+  private static final int MAX_SLEEP_TIME = 5 * 60;
 
   @Inject
-  public RMWebAppFilter(Injector injector) {
+  public RMWebAppFilter(Injector injector, Configuration conf) {
     super(injector);
     this.injector=injector;
+    InetSocketAddress sock = YarnConfiguration.useHttps(conf)
+        ? conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT)
+        : conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_WEBAPP_PORT);
+
+    path = sock.getHostName() + ":" + Integer.toString(sock.getPort());
+    path = YarnConfiguration.useHttps(conf)
+        ? "https://" + path
+        : "http://" + path;
   }
 
   @Override
@@ -69,9 +91,11 @@ public class RMWebAppFilter extends GuiceContainer {
     rmWebApp.checkIfStandbyRM();
     if (rmWebApp.isStandby()
         && shouldRedirect(rmWebApp, uri)) {
-      String redirectPath = rmWebApp.getRedirectPath() + uri;
+
+      String redirectPath = rmWebApp.getRedirectPath();
 
       if (redirectPath != null && !redirectPath.isEmpty()) {
+        redirectPath += uri;
         String redirectMsg =
             "This is standby RM. The redirect url is: " + redirectPath;
         PrintWriter out = response.getWriter();
@@ -79,11 +103,40 @@ public class RMWebAppFilter extends GuiceContainer {
         response.setHeader("Location", redirectPath);
         response.setStatus(HttpServletResponse.SC_TEMPORARY_REDIRECT);
         return;
+      } else {
+        boolean doRetry = true;
+        String retryIntervalStr =
+            request.getParameter(YarnWebParams.NEXT_REFRESH_INTERVAL);
+        int retryInterval = 0;
+        if (retryIntervalStr != null) {
+          try {
+            retryInterval = Integer.parseInt(retryIntervalStr.trim());
+          } catch (NumberFormatException ex) {
+            doRetry = false;
+          }
+        }
+        int next = calculateExponentialTime(retryInterval);
+
+        String redirectUrl =
+            appendOrReplaceParamter(path + uri,
+              YarnWebParams.NEXT_REFRESH_INTERVAL + "=" + (retryInterval + 1));
+        if (redirectUrl == null || next > MAX_SLEEP_TIME) {
+          doRetry = false;
+        }
+        String redirectMsg =
+            doRetry ? "Can not find any active RM. Will retry in next " + next
+                + " seconds." : "There is no active RM right now.";
+        PrintWriter out = response.getWriter();
+        out.println(redirectMsg);
+        if (doRetry) {
+          response.setHeader("Refresh", next + ";url=" + redirectUrl);
+          response.setStatus(HttpServletResponse.SC_TEMPORARY_REDIRECT);
+        }
       }
+      return;
     }
 
     super.doFilter(request, response, chain);
-
   }
 
   private boolean shouldRedirect(RMWebApp rmWebApp, String uri) {
@@ -92,4 +145,33 @@ public class RMWebAppFilter extends GuiceContainer {
         && !uri.startsWith(ProxyUriUtils.PROXY_BASE)
         && !NON_REDIRECTED_URIS.contains(uri);
   }
-}
+
+  private String appendOrReplaceParamter(String uri, String newQuery) {
+    if (uri.contains(YarnWebParams.NEXT_REFRESH_INTERVAL + "=")) {
+      return uri.replaceAll(YarnWebParams.NEXT_REFRESH_INTERVAL + "=[^&]+",
+        newQuery);
+    }
+    try {
+      URI oldUri = new URI(uri);
+      String appendQuery = oldUri.getQuery();
+      if (appendQuery == null) {
+        appendQuery = newQuery;
+      } else {
+        appendQuery += "&" + newQuery;
+      }
+
+      URI newUri =
+          new URI(oldUri.getScheme(), oldUri.getAuthority(), oldUri.getPath(),
+            appendQuery, oldUri.getFragment());
+
+      return newUri.toString();
+    } catch (URISyntaxException e) {
+      return null;
+    }
+  }
+
+  private static int calculateExponentialTime(int retries) {
+    long baseTime = BASIC_SLEEP_TIME * (1L << retries);
+    return (int) (baseTime * ((new Random()).nextDouble() + 0.5));
+  }
+}
\ No newline at end of file


[30/50] [abbrv] hadoop git commit: HADOOP-5323. Trash documentation should describe its directory structure and configurations. Contributed by Weiwei Yang.

Posted by wa...@apache.org.
HADOOP-5323. Trash documentation should describe its directory structure and configurations. 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/9a87f81c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9a87f81c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9a87f81c

Branch: refs/heads/YARN-1197
Commit: 9a87f81ce6839d1b5ab3d663c1e82c7e572bdeb8
Parents: b469ac5
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Thu Sep 3 22:34:25 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Thu Sep 3 22:34:25 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 3 +++
 .../hadoop-hdfs/src/site/markdown/HdfsDesign.md                | 6 ++++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a87f81c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 22b80a6..275beb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1297,6 +1297,9 @@ Release 2.7.2 - UNRELEASED
 
     HDFS-8659. Block scanner INFO message is spamming logs. (Yongjun Zhang)
 
+    HADOOP-5323. Trash documentation should describe its directory structure and
+    configurations. (Weiwei Yang via ozawa)
+
   OPTIMIZATIONS
 
     HDFS-8722. Optimize datanode writes for small writes and flushes (kihwal)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a87f81c/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
index c441ae8..5b1f66e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
@@ -226,9 +226,11 @@ Space Reclamation
 
 ### File Deletes and Undeletes
 
-When a file is deleted by a user or an application, it is not immediately removed from HDFS. Instead, HDFS first renames it to a file in the trash directory(`/user/<username>/.Trash`). The file can be restored quickly as long as it remains in trash. A file remains in trash for a configurable amount of time. After the expiry of its life in trash, the NameNode deletes the file from the HDFS namespace. The deletion of a file causes the blocks associated with the file to be freed. Note that there could be an appreciable time delay between the time a file is deleted by a user and the time of the corresponding increase in free space in HDFS.
+When a file is deleted by a user or an application, it is not immediately removed from HDFS. Instead, HDFS moves it to a trash directory (each user has its own trash directory under `/user/<username>/.Trash`).
+The file can be restored quickly as long as it remains in trash. Most recent deleted files are moved to the current trash directory (`/user/<username>/.Trash/Current`), and in a configurable interval, HDFS creates checkpoints (under `/user/<username>/.Trash/<date>`) for files in current trash directory and deletes old checkpoints when they are expired.
+After the expiry of its life in trash, the NameNode deletes the file from the HDFS namespace. The deletion of a file causes the blocks associated with the file to be freed. Note that there could be an appreciable time delay between the time a file is deleted by a user and the time of the corresponding increase in free space in HDFS.
 
-A user can Undelete a file after deleting it as long as it remains in the trash directory. If a user wants to undelete a file that he/she has deleted, he/she can navigate the trash directory and retrieve the file. The trash directory contains only the latest copy of the file that was deleted. The trash directory is just like any other directory with one special feature: HDFS applies specified policies to automatically delete files from this directory. Current default trash interval is set to 0 (Deletes file without storing in trash). This value is configurable parameter stored as `fs.trash.interval` stored in core-site.xml.
+Currently, the trash feature is disabled by default (deleting files without storing in trash). User can enable this feature by setting a value greater than zero for parameter `fs.trash.interval` (in core-site.xml). This value tells the NameNode how long a checkpoint will be expired and removed from HDFS. In addition, user can configure an appropriate time to tell NameNode how often to create checkpoints in trash (the parameter stored as `fs.trash.checkpoint.interval` in core-site.xml), this value should be smaller or equal to fs.trash.interval.
 
 ### Decrease Replication Factor
 


[44/50] [abbrv] hadoop git commit: YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: a38c1309dbc0f737ebf779ad88de77772450bd4a
Parents: d16c4ee
Author: Jian He <ji...@apache.org>
Authored: Mon Jul 13 17:34:26 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:02 2015 -0700

----------------------------------------------------------------------
 .../app/local/TestLocalContainerAllocator.java  |   6 +-
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../api/protocolrecords/AllocateRequest.java    |  57 ++++--
 .../api/protocolrecords/AllocateResponse.java   |  42 +++--
 .../records/ContainerResourceChangeRequest.java | 117 +++++++++++++
 .../api/records/ContainerResourceDecrease.java  |  78 ---------
 .../api/records/ContainerResourceIncrease.java  |  84 ---------
 .../ContainerResourceIncreaseRequest.java       |  80 ---------
 .../yarn/api/records/ContainerStatus.java       |  13 ++
 .../src/main/proto/yarn_protos.proto            |  14 +-
 .../src/main/proto/yarn_service_protos.proto    |  16 +-
 .../impl/pb/AllocateRequestPBImpl.java          | 119 ++++++++++---
 .../impl/pb/AllocateResponsePBImpl.java         | 175 +++++--------------
 .../ContainerResourceChangeRequestPBImpl.java   | 141 +++++++++++++++
 .../pb/ContainerResourceDecreasePBImpl.java     | 136 --------------
 .../pb/ContainerResourceIncreasePBImpl.java     | 171 ------------------
 .../ContainerResourceIncreaseRequestPBImpl.java | 141 ---------------
 .../records/impl/pb/ContainerStatusPBImpl.java  |  31 +++-
 .../hadoop/yarn/api/TestAllocateRequest.java    |  73 --------
 .../hadoop/yarn/api/TestAllocateResponse.java   | 114 ------------
 .../yarn/api/TestContainerResourceDecrease.java |  66 -------
 .../yarn/api/TestContainerResourceIncrease.java |  74 --------
 .../TestContainerResourceIncreaseRequest.java   |  68 -------
 .../hadoop/yarn/api/TestPBImplRecords.java      |  34 +---
 24 files changed, 534 insertions(+), 1319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
index f901ed8..167d804 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
@@ -46,8 +46,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -254,8 +252,8 @@ public class TestLocalContainerAllocator {
           Resources.none(), null, 1, null,
           Collections.<NMToken>emptyList(),
           yarnToken,
-          Collections.<ContainerResourceIncrease>emptyList(),
-          Collections.<ContainerResourceDecrease>emptyList());
+          Collections.<Container>emptyList(),
+          Collections.<Container>emptyList());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 662106b..60e2884 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -190,6 +190,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3970. Add REST api support for Application Priority.
     (Naganarasimha G R via vvasudev)
 
+    YARN-3866. AM-RM protocol changes to support container resizing. (Meng Ding
+    via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index 2458d9b..0b65e5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -22,11 +22,12 @@ import java.util.List;
 
 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.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -46,6 +47,16 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>
  *     A list of unused {@link Container} which are being returned.
  *   </li>
+ *   <li>
+ *     A list of {@link ContainerResourceChangeRequest} to inform
+ *     the <code>ResourceManager</code> about the resource increase
+ *     requirements of running containers.
+ *   </li>
+ *   <li>
+ *     A list of {@link ContainerResourceChangeRequest} to inform
+ *     the <code>ResourceManager</code> about the resource decrease
+ *     requirements of running containers.
+ *   </li>
  * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
@@ -61,7 +72,7 @@ public abstract class AllocateRequest {
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest) {
     return newInstance(responseID, appProgress, resourceAsk,
-        containersToBeReleased, resourceBlacklistRequest, null);
+        containersToBeReleased, resourceBlacklistRequest, null, null);
   }
   
   @Public
@@ -70,7 +81,8 @@ public abstract class AllocateRequest {
       List<ResourceRequest> resourceAsk,
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest,
-      List<ContainerResourceIncreaseRequest> increaseRequests) {
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     allocateRequest.setResponseId(responseID);
     allocateRequest.setProgress(appProgress);
@@ -78,6 +90,7 @@ public abstract class AllocateRequest {
     allocateRequest.setReleaseList(containersToBeReleased);
     allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
     allocateRequest.setIncreaseRequests(increaseRequests);
+    allocateRequest.setDecreaseRequests(decreaseRequests);
     return allocateRequest;
   }
   
@@ -184,20 +197,38 @@ public abstract class AllocateRequest {
       ResourceBlacklistRequest resourceBlacklistRequest);
   
   /**
-   * Get the <code>ContainerResourceIncreaseRequest</code> being sent by the
-   * <code>ApplicationMaster</code>
+   * Get the list of container resource increase requests being sent by the
+   * <code>ApplicationMaster</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceIncreaseRequest> getIncreaseRequests();
-  
+  @Unstable
+  public abstract List<ContainerResourceChangeRequest> getIncreaseRequests();
+
   /**
-   * Set the <code>ContainerResourceIncreaseRequest</code> to inform the
-   * <code>ResourceManager</code> about some container's resources need to be
-   * increased
+   * Set the list of container resource increase requests to inform the
+   * <code>ResourceManager</code> about the containers whose resources need
+   * to be increased.
    */
   @Public
-  @Stable
+  @Unstable
   public abstract void setIncreaseRequests(
-      List<ContainerResourceIncreaseRequest> increaseRequests);
+      List<ContainerResourceChangeRequest> increaseRequests);
+
+  /**
+   * Get the list of container resource decrease requests being sent by the
+   * <code>ApplicationMaster</code>.
+   */
+  @Public
+  @Unstable
+  public abstract List<ContainerResourceChangeRequest> getDecreaseRequests();
+
+  /**
+   * Set the list of container resource decrease requests to inform the
+   * <code>ResourceManager</code> about the containers whose resources need
+   * to be decreased.
+   */
+  @Public
+  @Unstable
+  public abstract void setDecreaseRequests(
+      List<ContainerResourceChangeRequest> decreaseRequests);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index c4fdb79..c363070 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -28,8 +28,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -59,6 +57,14 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>The number of available nodes in a cluster.</li>
  *   <li>A description of resources requested back by the cluster</li>
  *   <li>AMRMToken, if AMRMToken has been rolled over</li>
+ *   <li>
+ *     A list of {@link Container} representing the containers
+ *     whose resource has been increased.
+ *   </li>
+ *   <li>
+ *     A list of {@link Container} representing the containers
+ *     whose resource has been decreased.
+ *   </li>
  * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
@@ -94,8 +100,8 @@ public abstract class AllocateResponse {
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens,
-      List<ContainerResourceIncrease> increasedContainers,
-      List<ContainerResourceDecrease> decreasedContainers) {
+      List<Container> increasedContainers,
+      List<Container> decreasedContainers) {
     AllocateResponse response = newInstance(responseId, completedContainers,
         allocatedContainers, updatedNodes, availResources, command,
         numClusterNodes, preempt, nmTokens);
@@ -111,8 +117,8 @@ public abstract class AllocateResponse {
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
-      List<ContainerResourceIncrease> increasedContainers,
-      List<ContainerResourceDecrease> decreasedContainers) {
+      List<Container> increasedContainers,
+      List<Container> decreasedContainers) {
     AllocateResponse response =
         newInstance(responseId, completedContainers, allocatedContainers,
           updatedNodes, availResources, command, numClusterNodes, preempt,
@@ -263,34 +269,38 @@ public abstract class AllocateResponse {
   public abstract void setNMTokens(List<NMToken> nmTokens);
   
   /**
-   * Get the list of newly increased containers by <code>ResourceManager</code>
+   * Get the list of newly increased containers by
+   * <code>ResourceManager</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceIncrease> getIncreasedContainers();
+  @Unstable
+  public abstract List<Container> getIncreasedContainers();
 
   /**
-   * Set the list of newly increased containers by <code>ResourceManager</code>
+   * Set the list of newly increased containers by
+   * <code>ResourceManager</code>.
    */
   @Private
   @Unstable
   public abstract void setIncreasedContainers(
-      List<ContainerResourceIncrease> increasedContainers);
+      List<Container> increasedContainers);
 
   /**
-   * Get the list of newly decreased containers by <code>NodeManager</code>
+   * Get the list of newly decreased containers by
+   * <code>ResourceManager</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceDecrease> getDecreasedContainers();
+  @Unstable
+  public abstract List<Container> getDecreasedContainers();
 
   /**
-   * Set the list of newly decreased containers by <code>NodeManager</code>
+   * Set the list of newly decreased containers by
+   * <code>ResourceManager</code>.
    */
   @Private
   @Unstable
   public abstract void setDecreasedContainers(
-      List<ContainerResourceDecrease> decreasedContainers);
+      List<Container> decreasedContainers);
 
   /**
    * The AMRMToken that belong to this attempt

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java
new file mode 100644
index 0000000..117015b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.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.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code ContainerResourceChangeRequest} represents the request made by an
+ * application to the {@code ResourceManager} to change resource allocation of
+ * a running {@code Container}.
+ * <p>
+ * It includes:
+ * <ul>
+ *   <li>{@link ContainerId} for the container.</li>
+ *   <li>
+ *     {@link Resource} capability of the container after the resource change
+ *     is completed.
+ *   </li>
+ * </ul>
+ *
+ * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
+ */
+@Public
+@Unstable
+public abstract class ContainerResourceChangeRequest {
+
+  @Public
+  @Unstable
+  public static ContainerResourceChangeRequest newInstance(
+      ContainerId existingContainerId, Resource targetCapability) {
+    ContainerResourceChangeRequest context = Records
+        .newRecord(ContainerResourceChangeRequest.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    return context;
+  }
+
+  /**
+   * Get the <code>ContainerId</code> of the container.
+   * @return <code>ContainerId</code> of the container
+   */
+  @Public
+  @Unstable
+  public abstract ContainerId getContainerId();
+
+  /**
+   * Set the <code>ContainerId</code> of the container.
+   * @param containerId <code>ContainerId</code> of the container
+   */
+  @Public
+  @Unstable
+  public abstract void setContainerId(ContainerId containerId);
+
+  /**
+   * Get the <code>Resource</code> capability of the container.
+   * @return <code>Resource</code> capability of the container
+   */
+  @Public
+  @Unstable
+  public abstract Resource getCapability();
+
+  /**
+   * Set the <code>Resource</code> capability of the container.
+   * @param capability <code>Resource</code> capability of the container
+   */
+  @Public
+  @Unstable
+  public abstract void setCapability(Resource capability);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceChangeRequest) {
+      ContainerResourceChangeRequest ctx =
+          (ContainerResourceChangeRequest) other;
+
+      if (getContainerId() == null && ctx.getContainerId() != null) {
+        return false;
+      } else if (!getContainerId().equals(ctx.getContainerId())) {
+        return false;
+      }
+
+      if (getCapability() == null && ctx.getCapability() != null) {
+        return false;
+      } else if (!getCapability().equals(ctx.getCapability())) {
+        return false;
+      }
+
+      return true;
+    } else {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
deleted file mode 100644
index d766d92..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * Used by Application Master to ask Node Manager reduce size of a specified
- * container
- */
-public abstract class ContainerResourceDecrease {
-  @Public
-  public static ContainerResourceDecrease newInstance(
-      ContainerId existingContainerId, Resource targetCapability) {
-    ContainerResourceDecrease context = Records
-        .newRecord(ContainerResourceDecrease.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    return context;
-  }
-
-  @Public
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  public abstract void setCapability(Resource capability);
-  
-  @Override
-  public int hashCode() {
-    return getCapability().hashCode() + getContainerId().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof ContainerResourceDecrease) {
-      ContainerResourceDecrease ctx = (ContainerResourceDecrease)other;
-      
-      if (getContainerId() == null && ctx.getContainerId() != null) {
-        return false;
-      } else if (!getContainerId().equals(ctx.getContainerId())) {
-        return false;
-      }
-      
-      if (getCapability() == null && ctx.getCapability() != null) {
-        return false;
-      } else if (!getCapability().equals(ctx.getCapability())) {
-        return false;
-      }
-      
-      return true;
-    } else {
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
deleted file mode 100644
index f4c1560..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * Represent a new increased container accepted by Resource Manager
- */
-public abstract class ContainerResourceIncrease {
-  @Public
-  public static ContainerResourceIncrease newInstance(
-      ContainerId existingContainerId, Resource targetCapability, Token token) {
-    ContainerResourceIncrease context = Records
-        .newRecord(ContainerResourceIncrease.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    context.setContainerToken(token);
-    return context;
-  }
-
-  @Public
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  public abstract void setCapability(Resource capability);
-  
-  @Public
-  public abstract Token getContainerToken();
-
-  @Public
-  public abstract void setContainerToken(Token token);
-
-  @Override
-  public int hashCode() {
-    return getCapability().hashCode() + getContainerId().hashCode();
-  }
-  
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof ContainerResourceIncrease) {
-      ContainerResourceIncrease ctx = (ContainerResourceIncrease)other;
-      
-      if (getContainerId() == null && ctx.getContainerId() != null) {
-        return false;
-      } else if (!getContainerId().equals(ctx.getContainerId())) {
-        return false;
-      }
-      
-      if (getCapability() == null && ctx.getCapability() != null) {
-        return false;
-      } else if (!getCapability().equals(ctx.getCapability())) {
-        return false;
-      }
-      
-      return true;
-    } else {
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
deleted file mode 100644
index 9e3b640..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * Used by Application Master, send a container resource increase request to
- * Resource Manager
- */
-@Public
-public abstract class ContainerResourceIncreaseRequest {
-  @Public
-  public static ContainerResourceIncreaseRequest newInstance(
-      ContainerId existingContainerId, Resource targetCapability) {
-    ContainerResourceIncreaseRequest context = Records
-        .newRecord(ContainerResourceIncreaseRequest.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    return context;
-  }
-
-  @Public
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  public abstract void setCapability(Resource capability);
-
-  @Override
-  public int hashCode() {
-    return getCapability().hashCode() + getContainerId().hashCode();
-  }
-  
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof ContainerResourceIncreaseRequest) {
-      ContainerResourceIncreaseRequest ctx =
-          (ContainerResourceIncreaseRequest) other;
-      
-      if (getContainerId() == null && ctx.getContainerId() != null) {
-        return false;
-      } else if (!getContainerId().equals(ctx.getContainerId())) {
-        return false;
-      }
-      
-      if (getCapability() == null && ctx.getCapability() != null) {
-        return false;
-      } else if (!getCapability().equals(ctx.getCapability())) {
-        return false;
-      }
-      
-      return true;
-    } else {
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 5ccf6dc..2c2238f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>{@code ContainerState} of the container.</li>
  *   <li><em>Exit status</em> of a completed container.</li>
  *   <li><em>Diagnostic</em> message for a failed container.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
  * </ul>
  */
 @Public
@@ -114,4 +115,16 @@ public abstract class ContainerStatus {
   @Private
   @Unstable
   public abstract void setDiagnostics(String diagnostics);
+
+  /**
+   * Get the <code>Resource</code> allocated to the container.
+   * @return <code>Resource</code> allocated to the container
+   */
+  @Public
+  @Unstable
+  public abstract Resource getCapability();
+
+  @Private
+  @Unstable
+  public abstract void setCapability(Resource capability);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 1bd3dda..ee2f2bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -468,6 +468,7 @@ message ContainerStatusProto {
   optional ContainerStateProto state = 2;
   optional string diagnostics = 3 [default = "N/A"];
   optional int32 exit_status = 4 [default = -1000];
+  optional ResourceProto capability = 5;
 }
 
 enum ContainerExitStatusProto {
@@ -477,22 +478,11 @@ enum ContainerExitStatusProto {
   DISKS_FAILED = -101;
 }
 
-message ContainerResourceIncreaseRequestProto {
+message ContainerResourceChangeRequestProto {
   optional ContainerIdProto container_id = 1;
   optional ResourceProto capability = 2;
 } 
 
-message ContainerResourceIncreaseProto {
-  optional ContainerIdProto container_id = 1;
-  optional ResourceProto capability = 2;
-  optional hadoop.common.TokenProto container_token = 3;
-}
-
-message ContainerResourceDecreaseProto {
-  optional ContainerIdProto container_id = 1;
-  optional ResourceProto capability = 2;
-}
-
 ////////////////////////////////////////////////////////////////////////
 ////// From common//////////////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index b0b12d1..ff5a127 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -66,7 +66,8 @@ message AllocateRequestProto {
   optional ResourceBlacklistRequestProto blacklist_request = 3;
   optional int32 response_id = 4;
   optional float progress = 5;
-  repeated ContainerResourceIncreaseRequestProto increase_request = 6;
+  repeated ContainerResourceChangeRequestProto increase_request = 6;
+  repeated ContainerResourceChangeRequestProto decrease_request = 7;
 }
 
 message NMTokenProto {
@@ -84,8 +85,8 @@ message AllocateResponseProto {
   optional int32 num_cluster_nodes = 7;
   optional PreemptionMessageProto preempt = 8;
   repeated NMTokenProto nm_tokens = 9;
-  repeated ContainerResourceIncreaseProto increased_containers = 10;
-  repeated ContainerResourceDecreaseProto decreased_containers = 11;
+  repeated ContainerProto increased_containers = 10;
+  repeated ContainerProto decreased_containers = 11;
   optional hadoop.common.TokenProto am_rm_token = 12;
 }
 
@@ -286,6 +287,15 @@ message GetContainerStatusesResponseProto {
   repeated ContainerExceptionMapProto failed_requests = 2;
 }
 
+message IncreaseContainersResourceRequestProto {
+  repeated hadoop.common.TokenProto increase_containers = 1;
+}
+
+message IncreaseContainersResourceResponseProto {
+  repeated ContainerIdProto succeeded_requests = 1;
+  repeated ContainerExceptionMapProto failed_requests = 2;
+}
+
 //////////////////////////////////////////////////////
 /////// Application_History_Protocol /////////////////
 //////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
index dc11165..d6db32c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
@@ -27,15 +27,15 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
@@ -52,7 +52,8 @@ public class AllocateRequestPBImpl extends AllocateRequest {
 
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
-  private List<ContainerResourceIncreaseRequest> increaseRequests = null;
+  private List<ContainerResourceChangeRequest> increaseRequests = null;
+  private List<ContainerResourceChangeRequest> decreaseRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
   public AllocateRequestPBImpl() {
@@ -101,6 +102,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.increaseRequests != null) {
       addIncreaseRequestsToProto();
     }
+    if (this.decreaseRequests != null) {
+      addDecreaseRequestsToProto();
+    }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
@@ -162,14 +166,14 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
   
   @Override
-  public List<ContainerResourceIncreaseRequest> getIncreaseRequests() {
+  public List<ContainerResourceChangeRequest> getIncreaseRequests() {
     initIncreaseRequests();
     return this.increaseRequests;
   }
 
   @Override
   public void setIncreaseRequests(
-      List<ContainerResourceIncreaseRequest> increaseRequests) {
+      List<ContainerResourceChangeRequest> increaseRequests) {
     if (increaseRequests == null) {
       return;
     }
@@ -177,7 +181,24 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     this.increaseRequests.clear();
     this.increaseRequests.addAll(increaseRequests);
   }
-  
+
+  @Override
+  public List<ContainerResourceChangeRequest> getDecreaseRequests() {
+    initDecreaseRequests();
+    return this.decreaseRequests;
+  }
+
+  @Override
+  public void setDecreaseRequests(
+          List<ContainerResourceChangeRequest> decreaseRequests) {
+    if (decreaseRequests == null) {
+      return;
+    }
+    initDecreaseRequests();
+    this.decreaseRequests.clear();
+    this.decreaseRequests.addAll(decreaseRequests);
+  }
+
   @Override
   public ResourceBlacklistRequest getResourceBlacklistRequest() {
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
@@ -252,28 +273,42 @@ public class AllocateRequestPBImpl extends AllocateRequest {
       return;
     }
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceIncreaseRequestProto> list =
+    List<ContainerResourceChangeRequestProto> list =
         p.getIncreaseRequestList();
-    this.increaseRequests = new ArrayList<ContainerResourceIncreaseRequest>();
+    this.increaseRequests = new ArrayList<ContainerResourceChangeRequest>();
 
-    for (ContainerResourceIncreaseRequestProto c : list) {
+    for (ContainerResourceChangeRequestProto c : list) {
       this.increaseRequests.add(convertFromProtoFormat(c));
     }
   }
-  
+
+  private void initDecreaseRequests() {
+    if (this.decreaseRequests != null) {
+      return;
+    }
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerResourceChangeRequestProto> list =
+            p.getDecreaseRequestList();
+    this.decreaseRequests = new ArrayList<>();
+
+    for (ContainerResourceChangeRequestProto c : list) {
+      this.decreaseRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
   private void addIncreaseRequestsToProto() {
     maybeInitBuilder();
     builder.clearIncreaseRequest();
     if (increaseRequests == null) {
       return;
     }
-    Iterable<ContainerResourceIncreaseRequestProto> iterable =
-        new Iterable<ContainerResourceIncreaseRequestProto>() {
+    Iterable<ContainerResourceChangeRequestProto> iterable =
+        new Iterable<ContainerResourceChangeRequestProto>() {
           @Override
-          public Iterator<ContainerResourceIncreaseRequestProto> iterator() {
-            return new Iterator<ContainerResourceIncreaseRequestProto>() {
+          public Iterator<ContainerResourceChangeRequestProto> iterator() {
+            return new Iterator<ContainerResourceChangeRequestProto>() {
 
-              Iterator<ContainerResourceIncreaseRequest> iter =
+              Iterator<ContainerResourceChangeRequest> iter =
                   increaseRequests.iterator();
 
               @Override
@@ -282,7 +317,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
               }
 
               @Override
-              public ContainerResourceIncreaseRequestProto next() {
+              public ContainerResourceChangeRequestProto next() {
                 return convertToProtoFormat(iter.next());
               }
 
@@ -296,7 +331,43 @@ public class AllocateRequestPBImpl extends AllocateRequest {
         };
     builder.addAllIncreaseRequest(iterable);
   }
-  
+
+  private void addDecreaseRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearDecreaseRequest();
+    if (decreaseRequests == null) {
+      return;
+    }
+    Iterable<ContainerResourceChangeRequestProto> iterable =
+        new Iterable<ContainerResourceChangeRequestProto>() {
+          @Override
+          public Iterator<ContainerResourceChangeRequestProto> iterator() {
+            return new Iterator<ContainerResourceChangeRequestProto>() {
+
+              Iterator<ContainerResourceChangeRequest> iter =
+                      decreaseRequests.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ContainerResourceChangeRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllDecreaseRequest(iterable);
+  }
+
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -367,14 +438,14 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     return ((ResourceRequestPBImpl)t).getProto();
   }
   
-  private ContainerResourceIncreaseRequestPBImpl convertFromProtoFormat(
-      ContainerResourceIncreaseRequestProto p) {
-    return new ContainerResourceIncreaseRequestPBImpl(p);
+  private ContainerResourceChangeRequestPBImpl convertFromProtoFormat(
+      ContainerResourceChangeRequestProto p) {
+    return new ContainerResourceChangeRequestPBImpl(p);
   }
 
-  private ContainerResourceIncreaseRequestProto convertToProtoFormat(
-      ContainerResourceIncreaseRequest t) {
-    return ((ContainerResourceIncreaseRequestPBImpl) t).getProto();
+  private ContainerResourceChangeRequestProto convertToProtoFormat(
+      ContainerResourceChangeRequest t) {
+    return ((ContainerResourceChangeRequestPBImpl) t).getProto();
   }
 
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
index f2796fd..dd7d1a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
@@ -29,8 +29,6 @@ import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -38,8 +36,6 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
@@ -48,8 +44,6 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
@@ -72,8 +66,8 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   private List<Container> allocatedContainers = null;
   private List<NMToken> nmTokens = null;
   private List<ContainerStatus> completedContainersStatuses = null;
-  private List<ContainerResourceIncrease> increasedContainers = null;
-  private List<ContainerResourceDecrease> decreasedContainers = null;
+  private List<Container> increasedContainers = null;
+  private List<Container> decreasedContainers = null;
 
   private List<NodeReport> updatedNodes = null;
   private PreemptionMessage preempt;
@@ -147,14 +141,14 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
     if (this.increasedContainers != null) {
       builder.clearIncreasedContainers();
-      Iterable<ContainerResourceIncreaseProto> iterable =
-          getIncreaseProtoIterable(this.increasedContainers);
+      Iterable<ContainerProto> iterable =
+          getContainerProtoIterable(this.increasedContainers);
       builder.addAllIncreasedContainers(iterable);
     }
     if (this.decreasedContainers != null) {
       builder.clearDecreasedContainers();
-      Iterable<ContainerResourceDecreaseProto> iterable =
-          getChangeProtoIterable(this.decreasedContainers);
+      Iterable<ContainerProto> iterable =
+          getContainerProtoIterable(this.decreasedContainers);
       builder.addAllDecreasedContainers(iterable);
     }
     if (this.amrmToken != null) {
@@ -262,6 +256,36 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     allocatedContainers.addAll(containers);
   }
 
+  @Override
+  public synchronized List<Container> getIncreasedContainers() {
+    initLocalIncreasedContainerList();
+    return this.increasedContainers;
+  }
+
+  @Override
+  public synchronized void setIncreasedContainers(
+      final List<Container> containers) {
+    if (containers == null)
+      return;
+    initLocalIncreasedContainerList();
+    increasedContainers.addAll(containers);
+  }
+
+  @Override
+  public synchronized List<Container> getDecreasedContainers() {
+    initLocalDecreasedContainerList();
+    return this.decreasedContainers;
+  }
+
+  @Override
+  public synchronized void setDecreasedContainers(
+      final List<Container> containers) {
+    if (containers == null)
+      return;
+    initLocalDecreasedContainerList();
+    decreasedContainers.addAll(containers);
+  }
+
   //// Finished containers
   @Override
   public synchronized List<ContainerStatus> getCompletedContainersStatuses() {
@@ -333,37 +357,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   }
 
   @Override
-  public synchronized List<ContainerResourceIncrease> getIncreasedContainers() {
-    initLocalIncreasedContainerList();
-    return increasedContainers;
-  }
-
-  @Override
-  public synchronized void setIncreasedContainers(
-      List<ContainerResourceIncrease> increasedContainers) {
-    if (increasedContainers == null)
-      return;
-    initLocalIncreasedContainerList();
-    this.increasedContainers.addAll(increasedContainers);
-  }
-
-  @Override
-  public synchronized List<ContainerResourceDecrease> getDecreasedContainers() {
-    initLocalDecreasedContainerList();
-    return decreasedContainers;
-  }
-
-  @Override
-  public synchronized void setDecreasedContainers(
-      List<ContainerResourceDecrease> decreasedContainers) {
-    if (decreasedContainers == null) {
-      return;
-    }
-    initLocalDecreasedContainerList();
-    this.decreasedContainers.addAll(decreasedContainers);
-  }
-
-  @Override
   public synchronized Token getAMRMToken() {
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (amrmToken != null) {
@@ -390,10 +383,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       return;
     }
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceIncreaseProto> list = p.getIncreasedContainersList();
-    increasedContainers = new ArrayList<ContainerResourceIncrease>();
+    List<ContainerProto> list = p.getIncreasedContainersList();
+    increasedContainers = new ArrayList<>();
 
-    for (ContainerResourceIncreaseProto c : list) {
+    for (ContainerProto c : list) {
       increasedContainers.add(convertFromProtoFormat(c));
     }
   }
@@ -403,10 +396,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       return;
     }
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceDecreaseProto> list = p.getDecreasedContainersList();
-    decreasedContainers = new ArrayList<ContainerResourceDecrease>();
+    List<ContainerProto> list = p.getDecreasedContainersList();
+    decreasedContainers = new ArrayList<>();
 
-    for (ContainerResourceDecreaseProto c : list) {
+    for (ContainerProto c : list) {
       decreasedContainers.add(convertFromProtoFormat(c));
     }
   }
@@ -453,70 +446,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
   }
 
-  private synchronized Iterable<ContainerResourceIncreaseProto>
-      getIncreaseProtoIterable(
-          final List<ContainerResourceIncrease> newContainersList) {
-    maybeInitBuilder();
-    return new Iterable<ContainerResourceIncreaseProto>() {
-      @Override
-      public synchronized Iterator<ContainerResourceIncreaseProto> iterator() {
-        return new Iterator<ContainerResourceIncreaseProto>() {
-
-          Iterator<ContainerResourceIncrease> iter = newContainersList
-              .iterator();
-
-          @Override
-          public synchronized boolean hasNext() {
-            return iter.hasNext();
-          }
-
-          @Override
-          public synchronized ContainerResourceIncreaseProto next() {
-            return convertToProtoFormat(iter.next());
-          }
-
-          @Override
-          public synchronized void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-
-      }
-    };
-  }
-
-  private synchronized Iterable<ContainerResourceDecreaseProto>
-      getChangeProtoIterable(
-          final List<ContainerResourceDecrease> newContainersList) {
-    maybeInitBuilder();
-    return new Iterable<ContainerResourceDecreaseProto>() {
-      @Override
-      public synchronized Iterator<ContainerResourceDecreaseProto> iterator() {
-        return new Iterator<ContainerResourceDecreaseProto>() {
-
-          Iterator<ContainerResourceDecrease> iter = newContainersList
-              .iterator();
-
-          @Override
-          public synchronized boolean hasNext() {
-            return iter.hasNext();
-          }
-
-          @Override
-          public synchronized ContainerResourceDecreaseProto next() {
-            return convertToProtoFormat(iter.next());
-          }
-
-          @Override
-          public synchronized void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-
-      }
-    };
-  }
-  
   private synchronized Iterable<ContainerProto> getContainerProtoIterable(
       final List<Container> newContainersList) {
     maybeInitBuilder();
@@ -654,26 +583,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       completedContainersStatuses.add(convertFromProtoFormat(c));
     }
   }
-  
-  private synchronized ContainerResourceIncrease convertFromProtoFormat(
-      ContainerResourceIncreaseProto p) {
-    return new ContainerResourceIncreasePBImpl(p);
-  }
-
-  private synchronized ContainerResourceIncreaseProto convertToProtoFormat(
-      ContainerResourceIncrease t) {
-    return ((ContainerResourceIncreasePBImpl) t).getProto();
-  }
-
-  private synchronized ContainerResourceDecrease convertFromProtoFormat(
-      ContainerResourceDecreaseProto p) {
-    return new ContainerResourceDecreasePBImpl(p);
-  }
-
-  private synchronized ContainerResourceDecreaseProto convertToProtoFormat(
-      ContainerResourceDecrease t) {
-    return ((ContainerResourceDecreasePBImpl) t).getProto();
-  }
 
   private synchronized NodeReportPBImpl convertFromProtoFormat(
       NodeReportProto p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
new file mode 100644
index 0000000..f382b8c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
@@ -0,0 +1,141 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+
+public class ContainerResourceChangeRequestPBImpl extends
+    ContainerResourceChangeRequest {
+  ContainerResourceChangeRequestProto proto =
+      ContainerResourceChangeRequestProto.getDefaultInstance();
+  ContainerResourceChangeRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public ContainerResourceChangeRequestPBImpl() {
+    builder = ContainerResourceChangeRequestProto.newBuilder();
+  }
+
+  public ContainerResourceChangeRequestPBImpl(
+      ContainerResourceChangeRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceChangeRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.existingContainerId != null) {
+      return this.existingContainerId;
+    }
+    if (p.hasContainerId()) {
+      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
+    }
+    return this.existingContainerId;
+  }
+
+  @Override
+  public void setContainerId(ContainerId existingContainerId) {
+    maybeInitBuilder();
+    if (existingContainerId == null) {
+      builder.clearContainerId();
+    }
+    this.existingContainerId = existingContainerId;
+  }
+
+  @Override
+  public Resource getCapability() {
+    ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.targetCapability != null) {
+      return this.targetCapability;
+    }
+    if (p.hasCapability()) {
+      this.targetCapability = convertFromProtoFormat(p.getCapability());
+    }
+    return this.targetCapability;
+  }
+
+  @Override
+  public void setCapability(Resource targetCapability) {
+    maybeInitBuilder();
+    if (targetCapability == null) {
+      builder.clearCapability();
+    }
+    this.targetCapability = targetCapability;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private Resource convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerResourceChangeRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.existingContainerId != null) {
+      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
+    }
+    if (this.targetCapability != null) {
+      builder.setCapability(convertToProtoFormat(this.targetCapability));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
deleted file mode 100644
index 1834132..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records.impl.pb;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-
-public class ContainerResourceDecreasePBImpl extends ContainerResourceDecrease {
-  ContainerResourceDecreaseProto proto = ContainerResourceDecreaseProto
-      .getDefaultInstance();
-  ContainerResourceDecreaseProto.Builder builder = null;
-  boolean viaProto = false;
-
-  private ContainerId existingContainerId = null;
-  private Resource targetCapability = null;
-
-  public ContainerResourceDecreasePBImpl() {
-    builder = ContainerResourceDecreaseProto.newBuilder();
-  }
-
-  public ContainerResourceDecreasePBImpl(ContainerResourceDecreaseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public ContainerResourceDecreaseProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public ContainerId getContainerId() {
-    ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.existingContainerId != null) {
-      return this.existingContainerId;
-    }
-    if (p.hasContainerId()) {
-      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
-    }
-    return this.existingContainerId;
-  }
-
-  @Override
-  public void setContainerId(ContainerId existingContainerId) {
-    maybeInitBuilder();
-    if (existingContainerId == null) {
-      builder.clearContainerId();
-    }
-    this.existingContainerId = existingContainerId;
-  }
-
-  @Override
-  public Resource getCapability() {
-    ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.targetCapability != null) {
-      return this.targetCapability;
-    }
-    if (p.hasCapability()) {
-      this.targetCapability = convertFromProtoFormat(p.getCapability());
-    }
-    return this.targetCapability;
-  }
-
-  @Override
-  public void setCapability(Resource targetCapability) {
-    maybeInitBuilder();
-    if (targetCapability == null) {
-      builder.clearCapability();
-    }
-    this.targetCapability = targetCapability;
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl) t).getProto();
-  }
-
-  private Resource convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl) t).getProto();
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) {
-      maybeInitBuilder();
-    }
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = ContainerResourceDecreaseProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.existingContainerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
-    }
-    if (this.targetCapability != null) {
-      builder.setCapability(convertToProtoFormat(this.targetCapability));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
deleted file mode 100644
index 4e4f3a7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records.impl.pb;
-
-import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-
-public class ContainerResourceIncreasePBImpl extends ContainerResourceIncrease {
-  ContainerResourceIncreaseProto proto = ContainerResourceIncreaseProto
-      .getDefaultInstance();
-  ContainerResourceIncreaseProto.Builder builder = null;
-  boolean viaProto = false;
-
-  private ContainerId existingContainerId = null;
-  private Resource targetCapability = null;
-  private Token token = null;
-
-  public ContainerResourceIncreasePBImpl() {
-    builder = ContainerResourceIncreaseProto.newBuilder();
-  }
-
-  public ContainerResourceIncreasePBImpl(ContainerResourceIncreaseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public ContainerResourceIncreaseProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public ContainerId getContainerId() {
-    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.existingContainerId != null) {
-      return this.existingContainerId;
-    }
-    if (p.hasContainerId()) {
-      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
-    }
-    return this.existingContainerId;
-  }
-
-  @Override
-  public void setContainerId(ContainerId existingContainerId) {
-    maybeInitBuilder();
-    if (existingContainerId == null) {
-      builder.clearContainerId();
-    }
-    this.existingContainerId = existingContainerId;
-  }
-
-  @Override
-  public Resource getCapability() {
-    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.targetCapability != null) {
-      return this.targetCapability;
-    }
-    if (p.hasCapability()) {
-      this.targetCapability = convertFromProtoFormat(p.getCapability());
-    }
-    return this.targetCapability;
-  }
-
-  @Override
-  public void setCapability(Resource targetCapability) {
-    maybeInitBuilder();
-    if (targetCapability == null) {
-      builder.clearCapability();
-    }
-    this.targetCapability = targetCapability;
-  }
-  
-  @Override
-  public Token getContainerToken() {
-    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.token != null) {
-      return this.token;
-    }
-    if (p.hasContainerToken()) {
-      this.token = convertFromProtoFormat(p.getContainerToken());
-    }
-    return this.token;
-  }
-
-  @Override
-  public void setContainerToken(Token token) {
-    maybeInitBuilder();
-    if (token == null) {
-      builder.clearContainerToken();
-    }
-    this.token = token;
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl) t).getProto();
-  }
-
-  private Resource convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl) t).getProto();
-  }
-  
-  private Token convertFromProtoFormat(TokenProto p) {
-    return new TokenPBImpl(p);
-  }
-
-  private TokenProto convertToProtoFormat(Token t) {
-    return ((TokenPBImpl) t).getProto();
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) {
-      maybeInitBuilder();
-    }
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = ContainerResourceIncreaseProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.existingContainerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
-    }
-    if (this.targetCapability != null) {
-      builder.setCapability(convertToProtoFormat(this.targetCapability));
-    }
-    if (this.token != null) {
-      builder.setContainerToken(convertToProtoFormat(this.token));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
deleted file mode 100644
index f5ebf6c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records.impl.pb;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-
-
-public class ContainerResourceIncreaseRequestPBImpl extends
-    ContainerResourceIncreaseRequest {
-  ContainerResourceIncreaseRequestProto proto = 
-      ContainerResourceIncreaseRequestProto.getDefaultInstance();
-  ContainerResourceIncreaseRequestProto.Builder builder = null;
-  boolean viaProto = false;
-
-  private ContainerId existingContainerId = null;
-  private Resource targetCapability = null;
-
-  public ContainerResourceIncreaseRequestPBImpl() {
-    builder = ContainerResourceIncreaseRequestProto.newBuilder();
-  }
-
-  public ContainerResourceIncreaseRequestPBImpl(
-      ContainerResourceIncreaseRequestProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public ContainerResourceIncreaseRequestProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public ContainerId getContainerId() {
-    ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto
-        : builder;
-    if (this.existingContainerId != null) {
-      return this.existingContainerId;
-    }
-    if (p.hasContainerId()) {
-      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
-    }
-    return this.existingContainerId;
-  }
-
-  @Override
-  public void setContainerId(ContainerId existingContainerId) {
-    maybeInitBuilder();
-    if (existingContainerId == null) {
-      builder.clearContainerId();
-    }
-    this.existingContainerId = existingContainerId;
-  }
-
-  @Override
-  public Resource getCapability() {
-    ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto
-        : builder;
-    if (this.targetCapability != null) {
-      return this.targetCapability;
-    }
-    if (p.hasCapability()) {
-      this.targetCapability = convertFromProtoFormat(p.getCapability());
-    }
-    return this.targetCapability;
-  }
-
-  @Override
-  public void setCapability(Resource targetCapability) {
-    maybeInitBuilder();
-    if (targetCapability == null) {
-      builder.clearCapability();
-    }
-    this.targetCapability = targetCapability;
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl) t).getProto();
-  }
-
-  private Resource convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl) t).getProto();
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) {
-      maybeInitBuilder();
-    }
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = ContainerResourceIncreaseRequestProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.existingContainerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
-    }
-    if (this.targetCapability != null) {
-      builder.setCapability(convertToProtoFormat(this.targetCapability));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
index 86f2af9..d33d06d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
@@ -78,6 +80,7 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     sb.append("ContainerStatus: [");
     sb.append("ContainerId: ").append(getContainerId()).append(", ");
     sb.append("State: ").append(getState()).append(", ");
+    sb.append("Capability: ").append(getCapability()).append(", ");
     sb.append("Diagnostics: ").append(getDiagnostics()).append(", ");
     sb.append("ExitStatus: ").append(getExitStatus()).append(", ");
     sb.append("]");
@@ -168,6 +171,25 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     builder.setDiagnostics(diagnostics);
   }
 
+  @Override
+  public synchronized Resource getCapability() {
+    ContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasCapability()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getCapability());
+  }
+
+  @Override
+  public synchronized void setCapability(Resource capability) {
+    maybeInitBuilder();
+    if (capability == null) {
+      builder.clearCapability();
+      return;
+    }
+    builder.setCapability(convertToProtoFormat(capability));
+  }
+
   private ContainerStateProto convertToProtoFormat(ContainerState e) {
     return ProtoUtils.convertToProtoFormat(e);
   }
@@ -184,6 +206,11 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     return ((ContainerIdPBImpl)t).getProto();
   }
 
+  private ResourceProto convertToProtoFormat(Resource e) {
+    return ((ResourcePBImpl)e).getProto();
+  }
 
-
-}  
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
deleted file mode 100644
index 5ea29f8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
-import org.junit.Test;
-
-public class TestAllocateRequest {
-  @Test
-  public void testAllcoateRequestWithIncrease() {
-    List<ContainerResourceIncreaseRequest> incRequests =
-        new ArrayList<ContainerResourceIncreaseRequest>();
-    for (int i = 0; i < 3; i++) {
-      incRequests.add(ContainerResourceIncreaseRequest.newInstance(null,
-          Resource.newInstance(0, i)));
-    }
-    AllocateRequest r =
-        AllocateRequest.newInstance(123, 0f, null, null, null, incRequests);
-
-    // serde
-    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
-    r = new AllocateRequestPBImpl(p);
-
-    // check value
-    Assert.assertEquals(123, r.getResponseId());
-    Assert.assertEquals(incRequests.size(), r.getIncreaseRequests().size());
-
-    for (int i = 0; i < incRequests.size(); i++) {
-      Assert.assertEquals(r.getIncreaseRequests().get(i).getCapability()
-          .getVirtualCores(), incRequests.get(i).getCapability()
-          .getVirtualCores());
-    }
-  }
-
-  @Test
-  public void testAllcoateRequestWithoutIncrease() {
-    AllocateRequest r =
-        AllocateRequest.newInstance(123, 0f, null, null, null, null);
-
-    // serde
-    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
-    r = new AllocateRequestPBImpl(p);
-
-    // check value
-    Assert.assertEquals(123, r.getResponseId());
-    Assert.assertEquals(0, r.getIncreaseRequests().size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a38c1309/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
deleted file mode 100644
index fbe9af9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.AMCommand;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * 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.
- */
-public class TestAllocateResponse {
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testAllocateResponseWithIncDecContainers() {
-    List<ContainerResourceIncrease> incContainers =
-        new ArrayList<ContainerResourceIncrease>();
-    List<ContainerResourceDecrease> decContainers =
-        new ArrayList<ContainerResourceDecrease>();
-    for (int i = 0; i < 3; i++) {
-      incContainers.add(ContainerResourceIncrease.newInstance(null,
-          Resource.newInstance(1024, i), null));
-    }
-    for (int i = 0; i < 5; i++) {
-      decContainers.add(ContainerResourceDecrease.newInstance(null,
-          Resource.newInstance(1024, i)));
-    }
-
-    AllocateResponse r =
-        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
-            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
-            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(),
-            incContainers, decContainers);
-
-    // serde
-    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
-    r = new AllocateResponsePBImpl(p);
-
-    // check value
-    Assert
-        .assertEquals(incContainers.size(), r.getIncreasedContainers().size());
-    Assert
-        .assertEquals(decContainers.size(), r.getDecreasedContainers().size());
-
-    for (int i = 0; i < incContainers.size(); i++) {
-      Assert.assertEquals(i, r.getIncreasedContainers().get(i).getCapability()
-          .getVirtualCores());
-    }
-
-    for (int i = 0; i < decContainers.size(); i++) {
-      Assert.assertEquals(i, r.getDecreasedContainers().get(i).getCapability()
-          .getVirtualCores());
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testAllocateResponseWithoutIncDecContainers() {
-    AllocateResponse r =
-        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
-            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
-            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(), null, null);
-
-    // serde
-    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
-    r = new AllocateResponsePBImpl(p);
-
-    // check value
-    Assert.assertEquals(0, r.getIncreasedContainers().size());
-    Assert.assertEquals(0, r.getDecreasedContainers().size());
-  }
-}


[14/50] [abbrv] hadoop git commit: HADOOP-12359. hadoop fs -getmerge doc is wrong. Contributed by Jagadesh Kiran N.

Posted by wa...@apache.org.
HADOOP-12359. hadoop fs -getmerge doc is wrong. Contributed by Jagadesh Kiran N.


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

Branch: refs/heads/YARN-1197
Commit: 2e251a767427a38ecb6c309ad979feecb29a09f4
Parents: faa38e1
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 1 20:55:33 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 1 20:55:33 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt        |  3 +++
 .../hadoop-common/src/site/markdown/FileSystemShell.md | 13 +++++++++++--
 2 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e251a76/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 14e6fda..4eef964 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1120,6 +1120,9 @@ Release 2.7.2 - UNRELEASED
     HADOOP-12061. Incorrect command in single cluster setup document.
     (Kengo Seki via aajisaka)
 
+    HADOOP-12359. hadoop fs -getmerge doc is wrong.
+    (Jagadesh Kiran N via aajisaka)
+
 Release 2.7.1 - 2015-07-06 
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e251a76/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 fb89ca1..d6d00e4 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md
@@ -368,9 +368,18 @@ Returns 0 on success and non-zero on error.
 getmerge
 --------
 
-Usage: `hadoop fs -getmerge <src> <localdst> [addnl]`
+Usage: `hadoop fs -getmerge [-nl] <src> <localdst>`
 
-Takes a source directory and a destination file as input and concatenates files in src into the destination local file. Optionally addnl can be set to enable adding a newline character at the end of each file.
+Takes a source directory and a destination file as input and concatenates files in src into the destination local file. Optionally -nl can be set to enable adding a newline character (LF) at the end of each file.
+
+Examples:
+
+* `hadoop fs -getmerge -nl  /src  /opt/output.txt`
+* `hadoop fs -getmerge -nl  /src/file1.txt /src/file2.txt  /output.txt`
+
+Exit Code:
+
+Returns 0 on success and non-zero on error.
 
 help
 ----


[45/50] [abbrv] hadoop git commit: YARN-1449. AM-NM protocol changes to support container resizing. Contributed by Meng Ding & Wangda Tan)

Posted by wa...@apache.org.
YARN-1449. AM-NM protocol changes to support container resizing. Contributed by Meng Ding & Wangda Tan)


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

Branch: refs/heads/YARN-1197
Commit: 016da56a959b86fcc5c2864029bbe0b528becd31
Parents: a38c130
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 14 16:06:25 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:03 2015 -0700

----------------------------------------------------------------------
 .../v2/app/launcher/TestContainerLauncher.java  |  11 +
 .../app/launcher/TestContainerLauncherImpl.java |   9 +
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/api/ContainerManagementProtocol.java   |  30 ++-
 .../IncreaseContainersResourceRequest.java      |  75 ++++++
 .../IncreaseContainersResourceResponse.java     |  93 +++++++
 .../proto/containermanagement_protocol.proto    |   1 +
 ...ContainerManagementProtocolPBClientImpl.java |  20 ++
 ...ontainerManagementProtocolPBServiceImpl.java |  22 ++
 ...IncreaseContainersResourceRequestPBImpl.java | 170 +++++++++++++
 ...ncreaseContainersResourceResponsePBImpl.java | 241 +++++++++++++++++++
 .../hadoop/yarn/TestContainerLaunchRPC.java     |   8 +
 .../yarn/TestContainerResourceIncreaseRPC.java  | 162 +++++++++++++
 .../java/org/apache/hadoop/yarn/TestRPC.java    |   8 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |  20 ++
 .../containermanager/ContainerManagerImpl.java  |  13 +
 .../server/resourcemanager/NodeManager.java     |  11 +-
 .../resourcemanager/TestAMAuthorization.java    |   8 +
 .../TestApplicationMasterLauncher.java          |   9 +
 19 files changed, 910 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index 41ee65d..6c3a4d6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -30,6 +30,8 @@ import java.util.Map;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
@@ -449,5 +451,14 @@ public class TestContainerLauncher {
           "Dummy function cause"));
       throw new IOException(e);
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws IOException,
+        IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new IOException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
index 184f1b2..610448c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -454,6 +456,13 @@ public class TestContainerLauncherImpl {
     }
 
     @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException,
+        IOException {
+      return null;
+    }
+
+    @Override
     public void close() throws IOException {
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 60e2884..ed81a02 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -193,6 +193,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3866. AM-RM protocol changes to support container resizing. (Meng Ding
     via jianhe)
 
+    YARN-1449. AM-NM protocol changes to support container resizing.
+    (Meng Ding & Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
index 7aa43df..43e1d4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
@@ -22,6 +22,9 @@ import java.io.IOException;
 
 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.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -38,9 +41,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>The protocol between an <code>ApplicationMaster</code> and a 
- * <code>NodeManager</code> to start/stop containers and to get status
- * of running containers.</p>
- * 
+ * <code>NodeManager</code> to start/stop and increase resource of containers
+ * and to get status of running containers.</p>
+ *
  * <p>If security is enabled the <code>NodeManager</code> verifies that the
  * <code>ApplicationMaster</code> has truly been allocated the container
  * by the <code>ResourceManager</code> and also verifies all interactions such 
@@ -170,4 +173,25 @@ public interface ContainerManagementProtocol {
   GetContainerStatusesResponse getContainerStatuses(
       GetContainerStatusesRequest request) throws YarnException,
       IOException;
+
+  /**
+   * <p>
+   * The API used by the <code>ApplicationMaster</code> to request for
+   * resource increase of running containers on the <code>NodeManager</code>.
+   * </p>
+   *
+   * @param request
+   *         request to increase resource of a list of containers
+   * @return response which includes a list of containerIds of containers
+   *         whose resource has been successfully increased and a
+   *         containerId-to-exception map for failed requests.
+   *
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request) throws YarnException,
+      IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java
new file mode 100644
index 0000000..1fe8e94
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.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.yarn.api.protocolrecords;
+
+import java.util.List;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>The request sent by <code>Application Master</code> to the
+ * <code>Node Manager</code> to change the resource quota of a container.</p>
+ *
+ * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)
+ */
+@Public
+@Unstable
+public abstract class IncreaseContainersResourceRequest {
+  @Public
+  @Unstable
+  public static IncreaseContainersResourceRequest newInstance(
+      List<Token> containersToIncrease) {
+    IncreaseContainersResourceRequest request =
+        Records.newRecord(IncreaseContainersResourceRequest.class);
+    request.setContainersToIncrease(containersToIncrease);
+    return request;
+  }
+
+  /**
+   * Get a list of container tokens to be used for authorization during
+   * container resource increase.
+   * <p>
+   * Note: {@link NMToken} will be used for authenticating communication with
+   * {@code NodeManager}.
+   * @return the list of container tokens to be used for authorization during
+   * container resource increase.
+   * @see NMToken
+   */
+  @Public
+  @Unstable
+  public abstract List<Token> getContainersToIncrease();
+
+  /**
+   * Set container tokens to be used during container resource increase.
+   * The token is acquired from
+   * <code>AllocateResponse.getIncreasedContainers</code>.
+   * The token contains the container id and resource capability required for
+   * container resource increase.
+   * @param containersToIncrease the list of container tokens to be used
+   *                             for container resource increase.
+   */
+  @Public
+  @Unstable
+  public abstract void setContainersToIncrease(
+      List<Token> containersToIncrease);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
new file mode 100644
index 0000000..aeb1e83
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to increase container resource.
+ * </p>
+ *
+ * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)
+ */
+@Public
+@Unstable
+public abstract class IncreaseContainersResourceResponse {
+
+  @Private
+  @Unstable
+  public static IncreaseContainersResourceResponse newInstance(
+      List<ContainerId> successfullyIncreasedContainers,
+      Map<ContainerId, SerializedException> failedRequests) {
+    IncreaseContainersResourceResponse response =
+        Records.newRecord(IncreaseContainersResourceResponse.class);
+    response.setSuccessfullyIncreasedContainers(
+        successfullyIncreasedContainers);
+    response.setFailedRequests(failedRequests);
+    return response;
+  }
+
+  /**
+   * Get the list of containerIds of containers whose resource
+   * have been successfully increased.
+   *
+   * @return the list of containerIds of containers whose resource have
+   * been successfully increased.
+   */
+  @Public
+  @Unstable
+  public abstract List<ContainerId> getSuccessfullyIncreasedContainers();
+
+  /**
+   * Set the list of containerIds of containers whose resource have
+   * been successfully increased.
+   */
+  @Private
+  @Unstable
+  public abstract void setSuccessfullyIncreasedContainers(
+      List<ContainerId> succeedIncreasedContainers);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   */
+  @Public
+  @Unstable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
index 7b1647b..f06f6cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
@@ -34,4 +34,5 @@ service ContainerManagementProtocolService {
   rpc startContainers(StartContainersRequestProto) returns (StartContainersResponseProto);
   rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
   rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
+  rpc increaseContainersResource(IncreaseContainersResourceRequestProto) returns (IncreaseContainersResourceResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
index 15397e3..ce18bde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
@@ -30,12 +30,16 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
@@ -48,6 +52,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
 
 import com.google.protobuf.ServiceException;
 
@@ -128,4 +133,19 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
       return null;
     }
   }
+
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request) throws YarnException,
+      IOException {
+    IncreaseContainersResourceRequestProto requestProto =
+        ((IncreaseContainersResourceRequestPBImpl)request).getProto();
+    try {
+      return new IncreaseContainersResourceResponsePBImpl(
+          proxy.increaseContainersResource(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
index 2d33e69..7626441 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
@@ -23,9 +23,12 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
@@ -33,6 +36,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRespons
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
@@ -94,4 +99,21 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public IncreaseContainersResourceResponseProto increaseContainersResource(
+      RpcController controller, IncreaseContainersResourceRequestProto proto)
+      throws ServiceException {
+    IncreaseContainersResourceRequestPBImpl request =
+        new IncreaseContainersResourceRequestPBImpl(proto);
+    try {
+      IncreaseContainersResourceResponse response =
+          real.increaseContainersResource(request);
+      return ((IncreaseContainersResourceResponsePBImpl)response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
new file mode 100644
index 0000000..7417051
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
@@ -0,0 +1,170 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class IncreaseContainersResourceRequestPBImpl extends
+    IncreaseContainersResourceRequest {
+  IncreaseContainersResourceRequestProto proto =
+      IncreaseContainersResourceRequestProto.getDefaultInstance();
+  IncreaseContainersResourceRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<Token> containersToIncrease = null;
+
+  public IncreaseContainersResourceRequestPBImpl() {
+    builder = IncreaseContainersResourceRequestProto.newBuilder();
+  }
+
+  public IncreaseContainersResourceRequestPBImpl(
+      IncreaseContainersResourceRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public IncreaseContainersResourceRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containersToIncrease != null) {
+      addIncreaseContainersToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = IncreaseContainersResourceRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<Token> getContainersToIncrease() {
+    if (containersToIncrease != null) {
+      return containersToIncrease;
+    }
+    IncreaseContainersResourceRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<TokenProto> list = p.getIncreaseContainersList();
+    containersToIncrease = new ArrayList<>();
+    for (TokenProto c : list) {
+      containersToIncrease.add(convertFromProtoFormat(c));
+    }
+    return containersToIncrease;
+  }
+
+  @Override
+  public void setContainersToIncrease(List<Token> containersToIncrease) {
+    maybeInitBuilder();
+    if (containersToIncrease == null) {
+      builder.clearIncreaseContainers();
+    }
+    this.containersToIncrease = containersToIncrease;
+  }
+
+  private void addIncreaseContainersToProto() {
+    maybeInitBuilder();
+    builder.clearIncreaseContainers();
+    if (this.containersToIncrease == null) {
+      return;
+    }
+    Iterable<TokenProto> iterable = new Iterable<TokenProto>() {
+      @Override
+      public Iterator<TokenProto> iterator() {
+        return new Iterator<TokenProto>() {
+          Iterator<Token> iter = containersToIncrease.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public TokenProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllIncreaseContainers(iterable);
+  }
+
+  private Token convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
+  }
+
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
new file mode 100644
index 0000000..15062e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
@@ -0,0 +1,241 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class IncreaseContainersResourceResponsePBImpl extends
+    IncreaseContainersResourceResponse {
+  IncreaseContainersResourceResponseProto proto =
+      IncreaseContainersResourceResponseProto.getDefaultInstance();
+  IncreaseContainersResourceResponseProto.Builder builder = null;
+  boolean viaProto = false;
+  private List<ContainerId> succeededRequests = null;
+  private Map<ContainerId, SerializedException> failedRequests = null;
+
+  public IncreaseContainersResourceResponsePBImpl() {
+    builder = IncreaseContainersResourceResponseProto.newBuilder();
+  }
+
+  public IncreaseContainersResourceResponsePBImpl(
+      IncreaseContainersResourceResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public IncreaseContainersResourceResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.succeededRequests != null) {
+      addSucceededRequestsToProto();
+    }
+    if (this.failedRequests != null) {
+      addFailedRequestsToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = IncreaseContainersResourceResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<ContainerId> getSuccessfullyIncreasedContainers() {
+    initSucceededRequests();
+    return this.succeededRequests;
+  }
+
+  @Override
+  public void setSuccessfullyIncreasedContainers(
+      List<ContainerId> succeededRequests) {
+    maybeInitBuilder();
+    if (succeededRequests == null) {
+      builder.clearSucceededRequests();
+    }
+    this.succeededRequests = succeededRequests;
+  }
+
+  private void initSucceededRequests() {
+    if (this.succeededRequests != null) {
+      return;
+    }
+    IncreaseContainersResourceResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getSucceededRequestsList();
+    this.succeededRequests = new ArrayList<ContainerId>();
+    for (ContainerIdProto c : list) {
+      this.succeededRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void addSucceededRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSucceededRequests();
+    if (this.succeededRequests == null) {
+      return;
+    }
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+          Iterator<ContainerId> iter = succeededRequests.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllSucceededRequests(iterable);
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedRequests();
+    return this.failedRequests;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests) {
+    maybeInitBuilder();
+    if (failedRequests == null) {
+      builder.clearFailedRequests();
+    }
+    this.failedRequests = failedRequests;
+  }
+
+  private void initFailedRequests() {
+    if (this.failedRequests != null) {
+      return;
+    }
+    IncreaseContainersResourceResponseProtoOrBuilder
+        p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedRequests = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
+          convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  private void addFailedRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedRequests == null) {
+      return;
+    }
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
+        .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+          .setContainerId(convertToProtoFormat(entry.getKey()))
+          .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+          SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
index e2071dd..0a19783 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -166,5 +168,11 @@ public class TestContainerLaunchRPC {
           GetContainerStatusesResponse.newInstance(list, null);
       return null;
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
new file mode 100644
index 0000000..50ff1e0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.List;
+
+/*
+ * Test that the container resource increase rpc times out properly.
+ * This is used by AM to increase container resource.
+ */
+public class TestContainerResourceIncreaseRPC {
+
+  static final Log LOG = LogFactory.getLog(
+      TestContainerResourceIncreaseRPC.class);
+
+  @Test
+  public void testHadoopProtoRPCTimeout() throws Exception {
+    testRPCTimeout(HadoopYarnProtoRPC.class.getName());
+  }
+
+  private void testRPCTimeout(String rpcClass) throws Exception {
+    Configuration conf = new Configuration();
+    // set timeout low for the test
+    conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(ContainerManagementProtocol.class,
+        new DummyContainerManager(), addr, conf, null, 1);
+    server.start();
+    try {
+      ContainerManagementProtocol proxy =
+          (ContainerManagementProtocol) rpc.getProxy(
+              ContainerManagementProtocol.class,
+                  server.getListenerAddress(), conf);
+      ApplicationId applicationId = ApplicationId.newInstance(0, 0);
+      ApplicationAttemptId applicationAttemptId =
+          ApplicationAttemptId.newInstance(applicationId, 0);
+      ContainerId containerId =
+          ContainerId.newContainerId(applicationAttemptId, 100);
+      NodeId nodeId = NodeId.newInstance("localhost", 1234);
+      Resource resource = Resource.newInstance(1234, 2);
+      ContainerTokenIdentifier containerTokenIdentifier =
+          new ContainerTokenIdentifier(containerId, "localhost", "user",
+              resource, System.currentTimeMillis() + 10000, 42, 42,
+                  Priority.newInstance(0), 0);
+      Token containerToken =
+          TestRPC.newContainerToken(nodeId, "password".getBytes(),
+              containerTokenIdentifier);
+      // Construct container resource increase request,
+      List<Token> increaseTokens = new ArrayList<>();
+      increaseTokens.add(containerToken);
+      IncreaseContainersResourceRequest increaseRequest =
+          IncreaseContainersResourceRequest
+              .newInstance(increaseTokens);
+      try {
+        proxy.increaseContainersResource(increaseRequest);
+      } catch (Exception e) {
+        LOG.info(StringUtils.stringifyException(e));
+        Assert.assertEquals("Error, exception is not: "
+            + SocketTimeoutException.class.getName(),
+            SocketTimeoutException.class.getName(), e.getClass().getName());
+        return;
+      }
+    } finally {
+      server.stop();
+    }
+    Assert.fail("timeout exception should have occurred!");
+  }
+
+  public class DummyContainerManager implements ContainerManagementProtocol {
+
+    @Override
+    public StartContainersResponse startContainers(
+        StartContainersRequest requests) throws YarnException, IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public StopContainersResponse
+    stopContainers(StopContainersRequest requests) throws YarnException,
+        IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws YarnException, IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      try {
+        // make the thread sleep to look like its not going to respond
+        Thread.sleep(10000);
+      } catch (Exception e) {
+        LOG.error(e);
+        throw new YarnException(e);
+      }
+      throw new YarnException("Shouldn't happen!!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
index 39e6162..e718661 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
@@ -219,6 +221,12 @@ public class TestRPC {
           new Exception(EXCEPTION_CAUSE));
       throw new YarnException(e);
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 0d88bf4..d06d0a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -44,6 +44,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
@@ -101,6 +103,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequest
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -276,6 +280,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionReque
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
@@ -462,6 +468,8 @@ public class TestPBImplRecords {
     generateByNewInstance(ApplicationSubmissionContext.class);
     generateByNewInstance(ContainerReport.class);
     generateByNewInstance(ContainerResourceChangeRequest.class);
+    generateByNewInstance(IncreaseContainersResourceRequest.class);
+    generateByNewInstance(IncreaseContainersResourceResponse.class);
     generateByNewInstance(ContainerStatus.class);
     generateByNewInstance(PreemptionContainer.class);
     generateByNewInstance(PreemptionResourceRequest.class);
@@ -867,6 +875,18 @@ public class TestPBImplRecords {
   }
 
   @Test
+  public void testIncreaseContainersResourceRequestPBImpl() throws Exception {
+    validatePBImplRecord(IncreaseContainersResourceRequestPBImpl.class,
+        IncreaseContainersResourceRequestProto.class);
+  }
+
+  @Test
+  public void testIncreaseContainersResourceResponsePBImpl() throws Exception {
+    validatePBImplRecord(IncreaseContainersResourceResponsePBImpl.class,
+        IncreaseContainersResourceResponseProto.class);
+  }
+
+  @Test
   public void testSubmitApplicationRequestPBImpl() throws Exception {
     validatePBImplRecord(SubmitApplicationRequestPBImpl.class,
         SubmitApplicationRequestProto.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 68c7f2c..ce68b4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -60,6 +60,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -920,6 +922,17 @@ public class ContainerManagerImpl extends CompositeService implements
     return containerTokenIdentifier;
   }
 
+  /**
+   * Increase resource of a list of containers on this NodeManager.
+   */
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest requests)
+      throws YarnException, IOException {
+    // To be implemented in YARN-1645
+    return null;
+  }
+
   @Private
   @VisibleForTesting
   protected void updateNMTokenIdentifier(NMTokenIdentifier nmTokenIdentifier)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index d8d474e..5b7735e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -25,6 +25,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
@@ -295,7 +297,14 @@ public class NodeManager implements ContainerManagementProtocol {
     return GetContainerStatusesResponse.newInstance(statuses, null);
   }
 
-  public static org.apache.hadoop.yarn.server.api.records.NodeStatus 
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request)
+          throws YarnException, IOException {
+    return null;
+  }
+
+  public static org.apache.hadoop.yarn.server.api.records.NodeStatus
   createNodeStatus(NodeId nodeId, List<ContainerStatus> containers) {
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
     org.apache.hadoop.yarn.server.api.records.NodeStatus nodeStatus = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.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/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index c7f0d0a..2787f1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
@@ -122,6 +124,12 @@ public class TestAMAuthorization {
       return GetContainerStatusesResponse.newInstance(null, null);
     }
 
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(IncreaseContainersResourceRequest request)
+        throws YarnException {
+      return IncreaseContainersResourceResponse.newInstance(null, null);
+    }
+
     public Credentials getContainerCredentials() throws IOException {
       Credentials credentials = new Credentials();
       DataInputByteBuffer buf = new DataInputByteBuffer();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/016da56a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.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/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index d4f8e93..2760705 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -126,6 +128,13 @@ public class TestApplicationMasterLauncher {
         GetContainerStatusesRequest request) throws YarnException {
       return null;
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request)
+            throws YarnException {
+      return null;
+    }
   }
 
   @Test


[34/50] [abbrv] hadoop git commit: HDFS-8964. When validating the edit log, do not read at or beyond the file offset that is being written (Zhe Zhang via Colin P. McCabe)

Posted by wa...@apache.org.
HDFS-8964. When validating the edit log, do not read at or beyond the file offset that is being written (Zhe Zhang via Colin P. McCabe)


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

Branch: refs/heads/YARN-1197
Commit: 53c38cc89ab979ec47557dcfa7affbad20578c0a
Parents: 524ba87
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Sep 3 11:22:47 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Sep 3 11:22:47 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSEditLogTestUtil.java |   3 +-
 .../hadoop/hdfs/qjournal/server/Journal.java    |  22 ++--
 .../server/namenode/EditLogFileInputStream.java |  15 ++-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  10 ++
 .../hdfs/server/namenode/FSEditLogLoader.java   |  12 ++-
 .../server/namenode/FileJournalManager.java     |  39 +++++--
 .../hdfs/server/namenode/SecondaryNameNode.java |   2 +-
 .../TestCheckPointForSecurityTokens.java        |   4 +-
 .../hdfs/server/namenode/TestEditLog.java       | 103 ++++++++++++++++++-
 .../server/namenode/TestFSEditLogLoader.java    |  13 ++-
 11 files changed, 199 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 275dce2..afc6cf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1289,6 +1289,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9009. Send metrics logs to NullAppender by default. (Arpit Agarwal)
 
+    HDFS-8964. When validating the edit log, do not read at or beyond the file
+    offset that is being written (Zhe Zhang via Colin P. McCabe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java
index a46f9cf..e5b9d01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogTestUtil.java
@@ -33,7 +33,8 @@ public class FSEditLogTestUtil {
 
   public static long countTransactionsInStream(EditLogInputStream in) 
       throws IOException {
-    FSEditLogLoader.EditLogValidation validation = FSEditLogLoader.validateEditLog(in);
+    FSEditLogLoader.EditLogValidation validation =
+        FSEditLogLoader.validateEditLog(in, Long.MAX_VALUE);
     return (validation.getEndTxId() - in.getFirstTxId()) + 1;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 2953055..813f267 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -151,7 +151,7 @@ public class Journal implements Closeable {
     
     EditLogFile latest = scanStorageForLatestEdits();
     if (latest != null) {
-      highestWrittenTxId = latest.getLastTxId();
+      updateHighestWrittenTxId(latest.getLastTxId());
     }
   }
 
@@ -266,7 +266,17 @@ public class Journal implements Closeable {
   synchronized long getHighestWrittenTxId() {
     return highestWrittenTxId;
   }
-  
+
+  /**
+   * Update the highest Tx ID that has been written to the journal. Also update
+   * the {@link FileJournalManager#lastReadableTxId} of the underlying fjm.
+   * @param val The new value
+   */
+  private void updateHighestWrittenTxId(long val) {
+    highestWrittenTxId = val;
+    fjm.setLastReadableTxId(val);
+  }
+
   @VisibleForTesting
   JournalMetrics getMetricsForTests() {
     return metrics;
@@ -399,7 +409,7 @@ public class Journal implements Closeable {
     metrics.bytesWritten.incr(records.length);
     metrics.txnsWritten.incr(numTxns);
     
-    highestWrittenTxId = lastTxnId;
+    updateHighestWrittenTxId(lastTxnId);
     nextTxId = lastTxnId + 1;
   }
 
@@ -782,8 +792,8 @@ public class Journal implements Closeable {
             ": no current segment in place");
         
         // Update the highest txid for lag metrics
-        highestWrittenTxId = Math.max(segment.getEndTxId(),
-            highestWrittenTxId);
+        updateHighestWrittenTxId(Math.max(segment.getEndTxId(),
+            highestWrittenTxId));
       } else {
         LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) +
             ": old segment " + TextFormat.shortDebugString(currentSegment) +
@@ -812,7 +822,7 @@ public class Journal implements Closeable {
         // If we're shortening the log, update our highest txid
         // used for lag metrics.
         if (txnRange(currentSegment).containsLong(highestWrittenTxId)) {
-          highestWrittenTxId = segment.getEndTxId();
+          updateHighestWrittenTxId(segment.getEndTxId());
         }
       }
       syncedFile = syncLog(reqInfo, segment, fromUrl);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 73a162e..3bf0ab4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -300,8 +300,17 @@ public class EditLogFileInputStream extends EditLogInputStream {
     return getName();
   }
 
-  static FSEditLogLoader.EditLogValidation validateEditLog(File file)
-      throws IOException {
+  /**
+   * @param file File being validated.
+   * @param maxTxIdToValidate Maximum Tx ID to try to validate. Validation
+   *                          returns after reading this or a higher ID.
+   *                          The file portion beyond this ID is potentially
+   *                          being updated.
+   * @return Result of the validation
+   * @throws IOException
+   */
+  static FSEditLogLoader.EditLogValidation validateEditLog(File file,
+      long maxTxIdToValidate) throws IOException {
     EditLogFileInputStream in;
     try {
       in = new EditLogFileInputStream(file);
@@ -314,7 +323,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
     }
     
     try {
-      return FSEditLogLoader.validateEditLog(in);
+      return FSEditLogLoader.validateEditLog(in, maxTxIdToValidate);
     } finally {
       IOUtils.closeStream(in);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index edf88c9..e255cff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -674,6 +674,16 @@ public class FSEditLog implements LogsPurgeable {
       synchronized (this) {
         if (sync) {
           synctxid = syncStart;
+          for (JournalManager jm : journalSet.getJournalManagers()) {
+            /**
+             * {@link FileJournalManager#lastReadableTxId} is only meaningful
+             * for file-based journals. Therefore the interface is not added to
+             * other types of {@link JournalManager}.
+             */
+            if (jm instanceof FileJournalManager) {
+              ((FileJournalManager)jm).setLastReadableTxId(syncStart);
+            }
+          }
           isSyncRunning = false;
         }
         this.notifyAll();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index fc0bb78..bb36ca2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -1112,8 +1112,14 @@ public class FSEditLogLoader {
    * If there are invalid or corrupt transactions in the middle of the stream,
    * validateEditLog will skip over them.
    * This reads through the stream but does not close it.
+   *
+   * @param maxTxIdToValidate Maximum Tx ID to try to validate. Validation
+   *                          returns after reading this or a higher ID.
+   *                          The file portion beyond this ID is potentially
+   *                          being updated.
    */
-  static EditLogValidation validateEditLog(EditLogInputStream in) {
+  static EditLogValidation validateEditLog(EditLogInputStream in,
+      long maxTxIdToValidate) {
     long lastPos = 0;
     long lastTxId = HdfsServerConstants.INVALID_TXID;
     long numValid = 0;
@@ -1136,6 +1142,10 @@ public class FSEditLogLoader {
           || op.getTransactionId() > lastTxId) {
         lastTxId = op.getTransactionId();
       }
+      if (lastTxId >= maxTxIdToValidate) {
+        break;
+      }
+
       numValid++;
     }
     return new EditLogValidation(lastPos, lastTxId, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index ebd7475..a1488eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -76,6 +76,15 @@ public class FileJournalManager implements JournalManager {
 
   private File currentInProgress = null;
 
+  /**
+   * A FileJournalManager should maintain the largest Tx ID that has been
+   * safely written to its edit log files.
+   * It should limit readers to read beyond this ID to avoid potential race
+   * with ongoing writers.
+   * Initial value indicates that all transactions can be read.
+   */
+  private long lastReadableTxId = Long.MAX_VALUE;
+
   @VisibleForTesting
   StoragePurger purger
     = new NNStorageRetentionManager.DeletionStoragePurger();
@@ -159,6 +168,15 @@ public class FileJournalManager implements JournalManager {
     this.outputBufferCapacity = size;
   }
 
+
+  public long getLastReadableTxId() {
+    return lastReadableTxId;
+  }
+
+  public void setLastReadableTxId(long id) {
+    this.lastReadableTxId = id;
+  }
+
   @Override
   public void purgeLogsOlderThan(long minTxIdToKeep)
       throws IOException {
@@ -193,7 +211,7 @@ public class FileJournalManager implements JournalManager {
       }
       if (elf.isInProgress()) {
         try {
-          elf.validateLog();
+          elf.validateLog(getLastReadableTxId());
         } catch (IOException e) {
           LOG.error("got IOException while trying to validate header of " +
               elf + ".  Skipping.", e);
@@ -325,11 +343,13 @@ public class FileJournalManager implements JournalManager {
           (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
           "from among " + elfs.size() + " candidate file(s)");
     }
-    addStreamsToCollectionFromFiles(elfs, streams, fromTxId, inProgressOk);
+    addStreamsToCollectionFromFiles(elfs, streams, fromTxId,
+        getLastReadableTxId(), inProgressOk);
   }
   
   static void addStreamsToCollectionFromFiles(Collection<EditLogFile> elfs,
-      Collection<EditLogInputStream> streams, long fromTxId, boolean inProgressOk) {
+      Collection<EditLogInputStream> streams, long fromTxId, long maxTxIdToValidate,
+      boolean inProgressOk) {
     for (EditLogFile elf : elfs) {
       if (elf.isInProgress()) {
         if (!inProgressOk) {
@@ -340,7 +360,7 @@ public class FileJournalManager implements JournalManager {
           continue;
         }
         try {
-          elf.validateLog();
+          elf.validateLog(maxTxIdToValidate);
         } catch (IOException e) {
           LOG.error("got IOException while trying to validate header of " +
               elf + ".  Skipping.", e);
@@ -384,7 +404,7 @@ public class FileJournalManager implements JournalManager {
           continue;
         }
 
-        elf.validateLog();
+        elf.validateLog(getLastReadableTxId());
 
         if (elf.hasCorruptHeader()) {
           elf.moveAsideCorruptFile();
@@ -516,9 +536,14 @@ public class FileJournalManager implements JournalManager {
      * Find out where the edit log ends.
      * This will update the lastTxId of the EditLogFile or
      * mark it as corrupt if it is.
+     * @param maxTxIdToValidate Maximum Tx ID to try to validate. Validation
+     *                          returns after reading this or a higher ID.
+     *                          The file portion beyond this ID is potentially
+     *                          being updated.
      */
-    public void validateLog() throws IOException {
-      EditLogValidation val = EditLogFileInputStream.validateEditLog(file);
+    public void validateLog(long maxTxIdToValidate) throws IOException {
+      EditLogValidation val = EditLogFileInputStream.validateEditLog(file,
+          maxTxIdToValidate);
       this.lastTxId = val.getEndTxId();
       this.hasCorruptHeader = val.hasCorruptHeader();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index 2267853..e3e0a7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -907,7 +907,7 @@ public class SecondaryNameNode implements Runnable,
             throw new RuntimeException(ioe);
           }
           FileJournalManager.addStreamsToCollectionFromFiles(editFiles, streams,
-              fromTxId, inProgressOk);
+              fromTxId, Long.MAX_VALUE, inProgressOk);
         }
       }
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
index 9401d07..d5e64ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
@@ -88,7 +88,7 @@ public class TestCheckPointForSecurityTokens {
       for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
         EditLogFile log = FSImageTestUtil.findLatestEditsLog(sd);
         assertTrue(log.isInProgress());
-        log.validateLog();
+        log.validateLog(Long.MAX_VALUE);
         long numTransactions = (log.getLastTxId() - log.getFirstTxId()) + 1;
         assertEquals("In-progress log " + log + " should have 5 transactions",
                      5, numTransactions);;
@@ -105,7 +105,7 @@ public class TestCheckPointForSecurityTokens {
       for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
         EditLogFile log = FSImageTestUtil.findLatestEditsLog(sd);
         assertTrue(log.isInProgress());
-        log.validateLog();
+        log.validateLog(Long.MAX_VALUE);
         long numTransactions = (log.getLastTxId() - log.getFirstTxId()) + 1;
         assertEquals("In-progress log " + log + " should only have START txn",
             1, numTransactions);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index e59dec4..0495860 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -66,6 +66,8 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSInotifyEventInputStream;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -83,6 +85,9 @@ import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.spi.LoggingEvent;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.xml.sax.ContentHandler;
@@ -1223,7 +1228,8 @@ public class TestEditLog {
                                                                           TXNS_PER_ROLL*11);
 
     for (EditLogInputStream edits : editStreams) {
-      FSEditLogLoader.EditLogValidation val = FSEditLogLoader.validateEditLog(edits);
+      FSEditLogLoader.EditLogValidation val =
+          FSEditLogLoader.validateEditLog(edits, Long.MAX_VALUE);
       long read = (val.getEndTxId() - edits.getFirstTxId()) + 1;
       LOG.info("Loading edits " + edits + " read " + read);
       assertEquals(startTxId, edits.getFirstTxId());
@@ -1573,4 +1579,99 @@ public class TestEditLog {
       }
     }
   }
+
+  class TestAppender extends AppenderSkeleton {
+    private final List<LoggingEvent> log = new ArrayList<>();
+
+    @Override
+    public boolean requiresLayout() {
+      return false;
+    }
+
+    @Override
+    protected void append(final LoggingEvent loggingEvent) {
+      log.add(loggingEvent);
+    }
+
+    @Override
+    public void close() {
+    }
+
+    public List<LoggingEvent> getLog() {
+      return new ArrayList<>(log);
+    }
+  }
+
+  /**
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testReadActivelyUpdatedLog() throws Exception {
+    final TestAppender appender = new TestAppender();
+    LogManager.getRootLogger().addAppender(appender);
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    // Set single handler thread, so all transactions hit same thread-local ops.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY, 1);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      FSImage fsimage = cluster.getNamesystem().getFSImage();
+      StorageDirectory sd = fsimage.getStorage().getStorageDir(0);
+
+      final DistributedFileSystem fileSys = cluster.getFileSystem();
+      DFSInotifyEventInputStream events = fileSys.getInotifyEventStream();
+      fileSys.mkdirs(new Path("/test"));
+      fileSys.mkdirs(new Path("/test/dir1"));
+      fileSys.delete(new Path("/test/dir1"), true);
+      fsimage.getEditLog().logSync();
+      fileSys.mkdirs(new Path("/test/dir2"));
+
+
+      final File inProgressEdit = NNStorage.getInProgressEditsFile(sd, 1);
+      assertTrue(inProgressEdit.exists());
+      EditLogFileInputStream elis = new EditLogFileInputStream(inProgressEdit);
+      FSEditLogOp op;
+      long pos = 0;
+
+      while (true) {
+        op = elis.readOp();
+        if (op != null && op.opCode != FSEditLogOpCodes.OP_INVALID) {
+          pos = elis.getPosition();
+        } else {
+          break;
+        }
+      }
+      elis.close();
+      assertTrue(pos > 0);
+
+      RandomAccessFile rwf = new RandomAccessFile(inProgressEdit, "rw");
+      rwf.seek(pos);
+      assertEquals(rwf.readByte(), (byte) -1);
+
+      rwf.seek(pos + 1);
+      rwf.writeByte(2);
+
+      rwf.close();
+
+      events.poll();
+      String pattern = "Caught exception after reading (.*) ops";
+      Pattern r = Pattern.compile(pattern);
+      final List<LoggingEvent> log = appender.getLog();
+      for (LoggingEvent event : log) {
+        Matcher m = r.matcher(event.getRenderedMessage());
+        if (m.find()) {
+          fail("Should not try to read past latest syned edit log op");
+        }
+      }
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      LogManager.getRootLogger().removeAppender(appender);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c38cc8/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 55ba379..3c3423a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -318,7 +318,8 @@ public class TestFSEditLogLoader {
     } finally {
       rwf.close();
     }
-    EditLogValidation validation = EditLogFileInputStream.validateEditLog(logFile);
+    EditLogValidation validation =
+        EditLogFileInputStream.validateEditLog(logFile, Long.MAX_VALUE);
     assertTrue(validation.hasCorruptHeader());
   }
 
@@ -333,7 +334,7 @@ public class TestFSEditLogLoader {
     File logFileBak = new File(testDir, logFile.getName() + ".bak");
     Files.copy(logFile, logFileBak);
     EditLogValidation validation =
-        EditLogFileInputStream.validateEditLog(logFile);
+        EditLogFileInputStream.validateEditLog(logFile, Long.MAX_VALUE);
     assertTrue(!validation.hasCorruptHeader());
     // We expect that there will be an OP_START_LOG_SEGMENT, followed by
     // NUM_TXNS opcodes, followed by an OP_END_LOG_SEGMENT.
@@ -346,7 +347,8 @@ public class TestFSEditLogLoader {
       // Restore backup, corrupt the txn opcode
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, txOffset);
-      validation = EditLogFileInputStream.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile,
+          Long.MAX_VALUE);
       long expectedEndTxId = (txId == (NUM_TXNS + 1)) ?
           NUM_TXNS : (NUM_TXNS + 1);
       assertEquals("Failed when corrupting txn opcode at " + txOffset,
@@ -363,7 +365,8 @@ public class TestFSEditLogLoader {
       // Restore backup, corrupt the txn opcode
       Files.copy(logFileBak, logFile);
       truncateFile(logFile, txOffset);
-      validation = EditLogFileInputStream.validateEditLog(logFile);
+      validation = EditLogFileInputStream.validateEditLog(logFile,
+          Long.MAX_VALUE);
       long expectedEndTxId = (txId == 0) ?
           HdfsServerConstants.INVALID_TXID : (txId - 1);
       assertEquals("Failed when corrupting txid " + txId + " txn opcode " +
@@ -381,7 +384,7 @@ public class TestFSEditLogLoader {
     // layout flags section.
     truncateFile(logFile, 8);
     EditLogValidation validation =
-        EditLogFileInputStream.validateEditLog(logFile);
+        EditLogFileInputStream.validateEditLog(logFile, Long.MAX_VALUE);
     assertTrue(!validation.hasCorruptHeader());
     assertEquals(HdfsServerConstants.INVALID_TXID, validation.getEndTxId());
   }


[46/50] [abbrv] hadoop git commit: YARN-3867. ContainerImpl changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3867. ContainerImpl changes to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: 508da117e84f2c757aaa2a252d94291fa0b977b6
Parents: b05443e
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 28 13:51:23 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 4 11:48:03 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/server/utils/BuilderUtils.java  |  4 ++-
 .../containermanager/ContainerManagerImpl.java  |  7 ++--
 .../container/ChangeContainerResourceEvent.java | 36 -------------------
 .../containermanager/container/Container.java   |  2 ++
 .../container/ContainerEventType.java           |  4 ---
 .../container/ContainerImpl.java                | 16 ++++++---
 .../ChangeMonitoringContainerResourceEvent.java | 37 ++++++++++++++++++++
 .../monitor/ContainersMonitorEventType.java     |  3 +-
 .../nodemanager/metrics/NodeManagerMetrics.java | 11 ++++++
 .../nodemanager/TestNodeStatusUpdater.java      |  2 +-
 .../metrics/TestNodeManagerMetrics.java         | 18 +++++++---
 .../nodemanager/webapp/MockContainer.java       |  4 +++
 .../yarn/server/resourcemanager/MockNM.java     |  2 +-
 .../server/resourcemanager/NodeManager.java     |  2 +-
 .../resourcemanager/TestApplicationCleanup.java |  6 ++--
 .../attempt/TestRMAppAttemptTransitions.java    | 21 +++++++----
 .../capacity/TestCapacityScheduler.java         |  2 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  4 +--
 .../security/TestAMRMTokens.java                |  3 +-
 20 files changed, 118 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e17f413..49eb7b9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -199,6 +199,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1645. ContainerManager implementation to support container resizing.
     (Meng Ding & Wangda Tan via jianhe)
 
+    YARN-3867. ContainerImpl changes to support container resizing. (Meng Ding 
+    via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index a3bd6f8..475e9fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -200,13 +200,15 @@ public class BuilderUtils {
   }
 
   public static ContainerStatus newContainerStatus(ContainerId containerId,
-      ContainerState containerState, String diagnostics, int exitStatus) {
+      ContainerState containerState, String diagnostics, int exitStatus,
+      Resource capability) {
     ContainerStatus containerStatus = recordFactory
       .newRecordInstance(ContainerStatus.class);
     containerStatus.setState(containerState);
     containerStatus.setContainerId(containerId);
     containerStatus.setDiagnostics(diagnostics);
     containerStatus.setExitStatus(exitStatus);
+    containerStatus.setCapability(capability);
     return containerStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index e7572b5..6fddfa2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -115,7 +115,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ChangeContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
@@ -130,6 +129,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -1046,8 +1046,9 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        dispatcher.getEventHandler().handle(new ChangeContainerResourceEvent(
-            containerId, targetResource));
+        getContainersMonitor().handle(
+            new ChangeMonitoringContainerResourceEvent(
+                containerId, targetResource));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.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/container/ChangeContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
deleted file mode 100644
index 3944a3d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class ChangeContainerResourceEvent extends ContainerEvent {
-
-  private Resource resource;
-
-  public ChangeContainerResourceEvent(ContainerId c, Resource resource) {
-    super(c, ContainerEventType.CHANGE_CONTAINER_RESOURCE);
-    this.resource = resource;
-  }
-
-  public Resource getResource() {
-    return this.resource;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index 56b4fdd..1d2ec56 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -37,6 +37,8 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   Resource getResource();
 
+  void setResource(Resource targetResource);
+
   ContainerTokenIdentifier getContainerTokenIdentifier();
 
   String getUser();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.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/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index dc712bf..5622f8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -25,10 +25,6 @@ public enum ContainerEventType {
   KILL_CONTAINER,
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
-  CHANGE_CONTAINER_RESOURCE,
-
-  // Producer: ContainerMonitor
-  CONTAINER_RESOURCE_CHANGED,
 
   // DownloadManager
   CONTAINER_INITED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 3c76596..5c61a92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerImpl implements Container {
 
@@ -91,7 +92,7 @@ public class ContainerImpl implements Container {
   private final ContainerLaunchContext launchContext;
   private final ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
-  private final Resource resource;
+  private volatile Resource resource;
   private final String user;
   private int exitCode = ContainerExitStatus.INVALID;
   private final StringBuilder diagnostics;
@@ -249,7 +250,7 @@ public class ContainerImpl implements Container {
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
-        new KilledExternallyTransition()) 
+        new KilledExternallyTransition())
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -424,7 +425,7 @@ public class ContainerImpl implements Container {
     this.readLock.lock();
     try {
       return BuilderUtils.newContainerStatus(this.containerId,
-        getCurrentState(), diagnostics.toString(), exitCode);
+        getCurrentState(), diagnostics.toString(), exitCode, getResource());
     } finally {
       this.readLock.unlock();
     }
@@ -451,7 +452,14 @@ public class ContainerImpl implements Container {
 
   @Override
   public Resource getResource() {
-    return this.resource;
+    return Resources.clone(this.resource);
+  }
+
+  @Override
+  public void setResource(Resource targetResource) {
+    Resource currentResource = getResource();
+    this.resource = Resources.clone(targetResource);
+    this.metrics.changeContainer(currentResource, targetResource);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.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/monitor/ChangeMonitoringContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java
new file mode 100644
index 0000000..e0abbed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.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.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class ChangeMonitoringContainerResourceEvent extends ContainersMonitorEvent {
+  private final Resource resource;
+
+  public ChangeMonitoringContainerResourceEvent(ContainerId containerId,
+      Resource resource) {
+    super(containerId,
+        ContainersMonitorEventType.CHANGE_MONITORING_CONTAINER_RESOURCE);
+    this.resource = resource;
+  }
+
+  public Resource getResource() {
+    return this.resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.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/monitor/ContainersMonitorEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
index be99651..2b31480 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
@@ -20,5 +20,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
 public enum ContainersMonitorEventType {
   START_MONITORING_CONTAINER,
-  STOP_MONITORING_CONTAINER
+  STOP_MONITORING_CONTAINER,
+  CHANGE_MONITORING_CONTAINER_RESOURCE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.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/metrics/NodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
index 56797d1..a38d0b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
@@ -133,6 +133,17 @@ public class NodeManagerMetrics {
     availableVCores.incr(res.getVirtualCores());
   }
 
+  public void changeContainer(Resource before, Resource now) {
+    int deltaMB = now.getMemory() - before.getMemory();
+    int deltaVCores = now.getVirtualCores() - before.getVirtualCores();
+    allocatedMB = allocatedMB + deltaMB;
+    allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
+    availableMB = availableMB - deltaMB;
+    availableGB.set((int)Math.floor(availableMB/1024d));
+    allocatedVCores.incr(deltaVCores);
+    availableVCores.decr(deltaVCores);
+  }
+
   public void addResource(Resource res) {
     availableMB = availableMB + res.getMemory();
     availableGB.incr((int)Math.floor(availableMB/1024d));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index 3c0368b..70a8f55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -1662,7 +1662,7 @@ public class TestNodeStatusUpdater {
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(contaierId, containerState,
           "test_containerStatus: id=" + id + ", containerState: "
-              + containerState, 0);
+              + containerState, 0, Resource.newInstance(1024, 1));
     return containerStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.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/metrics/TestNodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
index 4dc4648..c0210d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
@@ -38,7 +38,12 @@ public class TestNodeManagerMetrics {
     Resource resource = Records.newRecord(Resource.class);
     resource.setMemory(512); //512MiB
     resource.setVirtualCores(2);
-
+    Resource largerResource = Records.newRecord(Resource.class);
+    largerResource.setMemory(1024);
+    largerResource.setVirtualCores(2);
+    Resource smallerResource = Records.newRecord(Resource.class);
+    smallerResource.setMemory(256);
+    smallerResource.setVirtualCores(1);
 
     metrics.addResource(total);
 
@@ -65,15 +70,20 @@ public class TestNodeManagerMetrics {
     metrics.initingContainer();
     metrics.runningContainer();
 
+    // Increase resource for a container
+    metrics.changeContainer(resource, largerResource);
+    // Decrease resource for a container
+    metrics.changeContainer(resource, smallerResource);
+
     Assert.assertTrue(!metrics.containerLaunchDuration.changed());
     metrics.addContainerLaunchDuration(1);
     Assert.assertTrue(metrics.containerLaunchDuration.changed());
 
     // availableGB is expected to be floored,
     // while allocatedGB is expected to be ceiled.
-    // allocatedGB: 3.5GB allocated memory is shown as 4GB
-    // availableGB: 4.5GB available memory is shown as 4GB
-    checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 14, 2);
+    // allocatedGB: 3.75GB allocated memory is shown as 4GB
+    // availableGB: 4.25GB available memory is shown as 4GB
+    checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 13, 3);
   }
 
   private void checkMetrics(int launched, int completed, int failed, int killed,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.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/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index b2ccb61..394a92c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -132,6 +132,10 @@ public class MockContainer implements Container {
   }
 
   @Override
+  public void setResource(Resource targetResource) {
+  }
+
+  @Override
   public ContainerTokenIdentifier getContainerTokenIdentifier() {
     return this.containerTokenIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.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/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index c917f79..4233cd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -143,7 +143,7 @@ public class MockNM {
         new HashMap<ApplicationId, List<ContainerStatus>>(1);
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
         BuilderUtils.newContainerId(attemptId, containerId), containerState,
-        "Success", 0);
+        "Success", 0, BuilderUtils.newResource(memory, vCores));
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>(1);
     containerStatusList.add(containerStatus);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index 5b7735e..b4ebf92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -193,7 +193,7 @@ public class NodeManager implements ContainerManagementProtocol {
 
       ContainerStatus containerStatus =
           BuilderUtils.newContainerStatus(container.getId(),
-            ContainerState.NEW, "", -1000);
+            ContainerState.NEW, "", -1000, container.getResource());
       applicationContainers.add(container);
       containerStatusMap.put(container, containerStatus);
       Resources.subtractFrom(available, tokenId.getResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/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 6e08aeb..3fa377e 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
@@ -231,7 +231,8 @@ public class TestApplicationCleanup {
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>();
     containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0)
-      .getId(), ContainerState.RUNNING, "nothing", 0));
+      .getId(), ContainerState.RUNNING, "nothing", 0,
+          conts.get(0).getResource()));
     containerStatuses.put(app.getApplicationId(), containerStatusList);
 
     NodeHeartbeatResponse resp = nm1.nodeHeartbeat(containerStatuses, true);
@@ -244,7 +245,8 @@ public class TestApplicationCleanup {
     containerStatuses.clear();
     containerStatusList.clear();
     containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0)
-      .getId(), ContainerState.RUNNING, "nothing", 0));
+      .getId(), ContainerState.RUNNING, "nothing", 0,
+          conts.get(0).getResource()));
     containerStatuses.put(app.getApplicationId(), containerStatusList);
 
     resp = nm1.nodeHeartbeat(containerStatuses, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.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/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index c8b6bd0..10ec453 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -956,7 +956,8 @@ public class TestRMAppAttemptTransitions {
     int exitCode = 123;
     ContainerStatus cs =
         BuilderUtils.newContainerStatus(amContainer.getId(),
-          ContainerState.COMPLETE, containerDiagMsg, exitCode);
+          ContainerState.COMPLETE, containerDiagMsg, exitCode,
+          amContainer.getResource());
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), cs, anyNodeId));
@@ -980,7 +981,8 @@ public class TestRMAppAttemptTransitions {
     String containerDiagMsg = "some error";
     int exitCode = 123;
     ContainerStatus cs = BuilderUtils.newContainerStatus(amContainer.getId(),
-        ContainerState.COMPLETE, containerDiagMsg, exitCode);
+        ContainerState.COMPLETE, containerDiagMsg, exitCode,
+            amContainer.getResource());
     ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
@@ -992,7 +994,8 @@ public class TestRMAppAttemptTransitions {
       applicationAttempt.getAppAttemptState());
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -1030,7 +1033,8 @@ public class TestRMAppAttemptTransitions {
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -1207,7 +1211,8 @@ public class TestRMAppAttemptTransitions {
             BuilderUtils.newContainerStatus(
                 BuilderUtils.newContainerId(
                     applicationAttempt.getAppAttemptId(), 42),
-                ContainerState.COMPLETE, "", 0), anyNodeId));
+                ContainerState.COMPLETE, "", 0,
+                    amContainer.getResource()), anyNodeId));
     testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl,
         diagnostics);
   }
@@ -1227,7 +1232,8 @@ public class TestRMAppAttemptTransitions {
         new RMAppAttemptContainerFinishedEvent(
             applicationAttempt.getAppAttemptId(),
             BuilderUtils.newContainerStatus(amContainer.getId(),
-                ContainerState.COMPLETE, "", 0), anyNodeId));
+                ContainerState.COMPLETE, "", 0,
+                    amContainer.getResource()), anyNodeId));
     testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
         diagnostics, 0, false);
   }
@@ -1256,7 +1262,8 @@ public class TestRMAppAttemptTransitions {
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
       applicationAttempt.getAppAttemptState());
     // send attempt_saved

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/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 44773be..5129100 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
@@ -856,7 +856,7 @@ public class TestCapacityScheduler {
     
     // Check container can complete successfully in case of resource over-commitment.
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
-        c1.getId(), ContainerState.COMPLETE, "", 0);
+        c1.getId(), ContainerState.COMPLETE, "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 6607211..83d2ccf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -746,7 +746,7 @@ public class TestFifoScheduler {
     Assert.assertEquals(GB, c1.getResource().getMemory());
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE,
-            "", 0);
+            "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) {
@@ -1141,7 +1141,7 @@ public class TestFifoScheduler {
     // over-commitment.
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE,
-            "", 0);
+            "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/508da117/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.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/TestAMRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
index 5dfd092..4488ad6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
@@ -171,7 +171,8 @@ public class TestAMRMTokens {
       ContainerStatus containerStatus =
           BuilderUtils.newContainerStatus(attempt.getMasterContainer().getId(),
               ContainerState.COMPLETE,
-              "AM Container Finished", 0);
+              "AM Container Finished", 0,
+              attempt.getMasterContainer().getResource());
       rm.getRMContext()
           .getDispatcher()
           .getEventHandler()


[40/50] [abbrv] hadoop git commit: YARN-4103. RM WebServices missing scheme for appattempts logLinks. Contributed by Jonathan Eagles.

Posted by wa...@apache.org.
YARN-4103. RM WebServices missing scheme for appattempts logLinks. 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/40d222e8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40d222e8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40d222e8

Branch: refs/heads/YARN-1197
Commit: 40d222e862063dc6c474cc6e8de0dce6c4395012
Parents: c83d13c
Author: Varun Vasudev <vv...@apache.org>
Authored: Thu Sep 3 23:58:26 2015 +0530
Committer: Varun Vasudev <vv...@apache.org>
Committed: Fri Sep 4 14:31:51 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../yarn/server/resourcemanager/webapp/RMAppBlock.java    |  3 ++-
 .../yarn/server/resourcemanager/webapp/RMWebServices.java |  7 ++++---
 .../server/resourcemanager/webapp/dao/AppAttemptInfo.java | 10 +++++-----
 .../resourcemanager/webapp/TestRMWebServicesApps.java     |  2 +-
 5 files changed, 15 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40d222e8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 4e9b411..29eabb5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -877,6 +877,9 @@ Release 2.7.2 - UNRELEASED
     YARN-4101. RM should print alert messages if Zookeeper and Resourcemanager
     gets connection issue. (Xuan Gong via jianhe)
 
+    YARN-4103. RM WebServices missing scheme for appattempts logLinks.
+    (Jonathan Eagles via vvasudeb)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40d222e8/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 38e0e3b..eec1fad 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
@@ -130,7 +130,8 @@ public class RMAppBlock extends AppBlock{
         continue;
       }
       AppAttemptInfo attemptInfo =
-          new AppAttemptInfo(this.rm, rmAppAttempt, rmApp.getUser());
+          new AppAttemptInfo(this.rm, rmAppAttempt, rmApp.getUser(),
+              WebAppUtils.getHttpSchemePrefix(conf));
       String blacklistedNodesCount = "N/A";
       Set<String> nodes =
           RMAppAttemptBlock.getBlacklistedNodes(rm,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40d222e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 8e11e2a..2410053 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -696,7 +696,8 @@ public class RMWebServices {
   @GET
   @Path("/apps/{appid}/appattempts")
   @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public AppAttemptsInfo getAppAttempts(@PathParam("appid") String appId) {
+  public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest hsr,
+      @PathParam("appid") String appId) {
 
     init();
     if (appId == null || appId.isEmpty()) {
@@ -714,8 +715,8 @@ public class RMWebServices {
 
     AppAttemptsInfo appAttemptsInfo = new AppAttemptsInfo();
     for (RMAppAttempt attempt : app.getAppAttempts().values()) {
-      AppAttemptInfo attemptInfo =
-          new AppAttemptInfo(rm, attempt, app.getUser());
+      AppAttemptInfo attemptInfo = new AppAttemptInfo(rm, attempt,
+          app.getUser(), hsr.getScheme() + "://");
       appAttemptsInfo.add(attemptInfo);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40d222e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
index 2c7bbd0..b6e95a6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMAppAttemptBlock;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
@@ -46,7 +45,8 @@ public class AppAttemptInfo {
   public AppAttemptInfo() {
   }
 
-  public AppAttemptInfo(ResourceManager rm, RMAppAttempt attempt, String user) {
+  public AppAttemptInfo(ResourceManager rm, RMAppAttempt attempt, String user,
+      String schemePrefix) {
     this.startTime = 0;
     this.containerId = "";
     this.nodeHttpAddress = "";
@@ -61,9 +61,9 @@ public class AppAttemptInfo {
         this.containerId = masterContainer.getId().toString();
         this.nodeHttpAddress = masterContainer.getNodeHttpAddress();
         this.nodeId = masterContainer.getNodeId().toString();
-        this.logsLink =
-            WebAppUtils.getRunningLogURL("//" + masterContainer.getNodeHttpAddress(),
-                ConverterUtils.toString(masterContainer.getId()), user);
+        this.logsLink = WebAppUtils.getRunningLogURL(schemePrefix
+            + masterContainer.getNodeHttpAddress(),
+            ConverterUtils.toString(masterContainer.getId()), user);
         if (rm.getResourceScheduler() instanceof AbstractYarnScheduler) {
           AbstractYarnScheduler ayScheduler =
               (AbstractYarnScheduler) rm.getResourceScheduler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40d222e8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.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/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index 919bb1a..a784295 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -1635,7 +1635,7 @@ public class TestRMWebServicesApps extends JerseyTestBase {
         .getMasterContainer().getNodeHttpAddress(), nodeHttpAddress);
     WebServicesTestUtils.checkStringMatch("nodeId", appAttempt
         .getMasterContainer().getNodeId().toString(), nodeId);
-    assertTrue("logsLink doesn't match", logsLink.startsWith("//"));
+    assertTrue("logsLink doesn't match ", logsLink.startsWith("http://"));
     assertTrue(
         "logsLink doesn't contain user info", logsLink.endsWith("/"
         + user));